[GitHub] flink pull request #2976: [FLINK-5303] [table] Support for SQL GROUPING SETS...

2016-12-08 Thread chermenin
GitHub user chermenin opened a pull request:

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

[FLINK-5303] [table] Support for SQL GROUPING SETS clause.

Support for operators GROUPING SETS / ROLLUP / CUBE was added in this PR.
Also added some tests for check execution of SQL queries with them.
PR will close next issue: https://issues.apache.org/jira/browse/FLINK-5303.

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

$ git pull https://github.com/chermenin/flink flink-5303

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

https://github.com/apache/flink/pull/2976.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 #2976


commit 51832104b5bb9aac06b6b86c98944a2d512e358c
Author: Aleksandr Chermenin 
Date:   2016-12-07T07:57:04Z

[FLINK-5303] Added GROUPING SETS implementation.

commit 9594a197148b77ffd4873d6fb77efafe01915c6e
Author: Aleksandr Chermenin 
Date:   2016-12-07T14:23:35Z

[FLINK-5303] Fixed grouping sets implementation.

commit a1aa9b2315974e63fee4f948b0e99580c49413ab
Author: Aleksandr Chermenin 
Date:   2016-12-07T14:35:46Z

[FLINK-5303] Small fixes.

commit c1170e2ce6111a77d31d29fbbad6b2c660d9e980
Author: Aleksandr Chermenin 
Date:   2016-12-08T07:46:09Z

[FLINK-5303] Some improvements.

commit 400c78d4b78fd092da0756177fa7e6dcfe7544b8
Author: Aleksandr Chermenin 
Date:   2016-12-08T09:32:53Z

[FLINK-5303] Added tests.

commit 8f30cbadca6a610ae9b4894065c4af38ec7ab12d
Author: Aleksandr Chermenin 
Date:   2016-12-08T09:34:35Z

[FLINK-5303] Test small fix.

commit eaa745bb907695bc70a0b61bc4e322ad617cd1b1
Author: Aleksandr Chermenin 
Date:   2016-12-08T11:34:19Z

[FLINK-5303] Grouping sets tests and fixes.

commit 543b2be72ec30f6fce2a25371cc0b8b95a49f832
Author: Aleksandr Chermenin 
Date:   2016-12-08T11:44:41Z

[FLINK-5303] Some cleanup.

commit 3976cea7ce3ad98381e6467d6cf1e02f1d19b103
Author: Aleksandr Chermenin 
Date:   2016-12-08T13:14:14Z

[FLINK-5303] Have supplemented documentation.

commit 92955c58fc464be34f3e3af0a83d38a6261edca3
Author: Aleksandr Chermenin 
Date:   2016-12-08T14:56:00Z

[FLINK-5303] Improved documentation.




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #2976: [FLINK-5303] [table] Support for SQL GROUPING SETS...

2016-12-28 Thread KurtYoung
Github user KurtYoung commented on a diff in the pull request:

https://github.com/apache/flink/pull/2976#discussion_r94104637
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamAggregate.scala
 ---
@@ -70,7 +71,9 @@ class DataStreamAggregate(
   namedAggregates,
   getRowType,
   inputType,
-  grouping)
+  grouping,
+  indicator
+)
--- End diff --

No need to have new line here.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #2976: [FLINK-5303] [table] Support for SQL GROUPING SETS...

2016-12-28 Thread KurtYoung
Github user KurtYoung commented on a diff in the pull request:

https://github.com/apache/flink/pull/2976#discussion_r94104624
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetAggregate.scala
 ---
@@ -104,7 +107,9 @@ class DataSetAggregate(
   namedAggregates,
   inputType,
   rowRelDataType,
-  grouping)
+  grouping,
+  indicator
+)
--- End diff --

No need to have new line here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #2976: [FLINK-5303] [table] Support for SQL GROUPING SETS...

2016-12-28 Thread KurtYoung
Github user KurtYoung commented on a diff in the pull request:

https://github.com/apache/flink/pull/2976#discussion_r94104657
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamAggregate.scala
 ---
@@ -207,7 +210,9 @@ class DataStreamAggregate(
 inputType,
 rowRelDataType,
 grouping,
-namedProperties)
+indicator,
+namedProperties
+  )
--- End diff --

Same as here and a couple of other places


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #2976: [FLINK-5303] [table] Support for SQL GROUPING SETS...

2016-12-28 Thread KurtYoung
Github user KurtYoung commented on a diff in the pull request:

https://github.com/apache/flink/pull/2976#discussion_r94104766
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateReduceCombineFunction.scala
 ---
@@ -41,12 +41,14 @@ class AggregateReduceCombineFunction(
 private val aggregates: Array[Aggregate[_ <: Any]],
 private val groupKeysMapping: Array[(Int, Int)],
 private val aggregateMapping: Array[(Int, Int)],
+private val additionalMapping: Array[(Int, Int)],
--- End diff --

Can we have a more explicit name about "what is the additional mapping?". 
And we should update the class document.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #2976: [FLINK-5303] [table] Support for SQL GROUPING SETS...

2016-12-28 Thread KurtYoung
Github user KurtYoung commented on a diff in the pull request:

https://github.com/apache/flink/pull/2976#discussion_r94104787
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateReduceGroupFunction.scala
 ---
@@ -40,6 +40,7 @@ class AggregateReduceGroupFunction(
 private val aggregates: Array[Aggregate[_ <: Any]],
 private val groupKeysMapping: Array[(Int, Int)],
 private val aggregateMapping: Array[(Int, Int)],
+private val additionalMapping: Array[(Int, Int)],
--- End diff --

Please update the class document


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #2976: [FLINK-5303] [table] Support for SQL GROUPING SETS...

2016-12-28 Thread KurtYoung
Github user KurtYoung commented on a diff in the pull request:

https://github.com/apache/flink/pull/2976#discussion_r94104843
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
 ---
@@ -98,7 +98,9 @@ object AggregateUtil {
 namedAggregates: Seq[CalcitePair[AggregateCall, String]],
 inputType: RelDataType,
 outputType: RelDataType,
-groupings: Array[Int]): RichGroupReduceFunction[Row, Row] = {
+groupings: Array[Int],
+indicator: Boolean
--- End diff --

This variable is not used 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #2976: [FLINK-5303] [table] Support for SQL GROUPING SETS...

2017-01-06 Thread twalthr
Github user twalthr commented on a diff in the pull request:

https://github.com/apache/flink/pull/2976#discussion_r94969330
  
--- Diff: 
flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/sql/GroupingSetsTest.java
 ---
@@ -0,0 +1,188 @@
+/*
+ * 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.api.java.batch.sql;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.operators.MapOperator;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.java.BatchTableEnvironment;
+import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
+import org.apache.flink.test.util.TestBaseUtils;
+import org.apache.flink.types.Row;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Comparator;
+import java.util.List;
+
+public class GroupingSetsTest {
+
+   private final static String TABLE_NAME = "MyTable";
+   private final static String TABLE_WITH_NULLS_NAME = "MyTableWithNulls";
+   private BatchTableEnvironment tableEnv;
+
+   @Before
+   public void setup() {
+   ExecutionEnvironment env = 
ExecutionEnvironment.getExecutionEnvironment();
+   tableEnv = TableEnvironment.getTableEnvironment(env, new 
TableConfig());
+
+   DataSet> dataSet = 
CollectionDataSets.get3TupleDataSet(env);
+   tableEnv.registerDataSet(TABLE_NAME, dataSet);
+
+   MapOperator, Tuple3> dataSetWithNulls =
+   dataSet.map(new MapFunction, Tuple3>() {
+
+   @Override
+   public Tuple3 
map(Tuple3 value) throws Exception {
+   if 
(value.f2.toLowerCase().contains("world")) {
+   value.f2 = null;
+   }
+   return value;
+   }
+   });
+   tableEnv.registerDataSet(TABLE_WITH_NULLS_NAME, 
dataSetWithNulls);
+   }
+
+   @Test
+   public void testGroupingSets() throws Exception {
+   String query =
+   "SELECT f1, f2, avg(f0) as a, GROUP_ID() as g FROM " + 
TABLE_NAME +
+   " GROUP BY GROUPING SETS (f1, f2)";
+
+   String expected =
+   
"6,null,18,1\n5,null,13,1\n4,null,8,1\n3,null,5,1\n2,null,2,1\n1,null,1,1\n" +
+   "null,Luke Skywalker,6,2\nnull,I am 
fine.,5,2\nnull,Hi,1,2\n" +
+   "null,Hello world, how are you?,4,2\nnull,Hello 
world,3,2\nnull,Hello,2,2\n" +
+   
"null,Comment#9,15,2\nnull,Comment#8,14,2\nnull,Comment#7,13,2\n" +
+   
"null,Comment#6,12,2\nnull,Comment#5,11,2\nnull,Comment#4,10,2\n" +
+   
"null,Comment#3,9,2\nnull,Comment#2,8,2\nnull,Comment#15,21,2\n" +
+   
"null,Comment#14,20,2\nnull,Comment#13,19,2\nnull,Comment#12,18,2\n" +
+   
"null,Comment#11,17,2\nnull,Comment#10,16,2\nnull,Comment#1,7,2";
+
+   checkSql(query, expected);
+   }
+
+   @Test
+   public void testGroupingSetsWithNulls() throws Exception {
--- End diff --

Can you add `GROUPING()` and `GROUPING_ID()` here too. Just to make sure 
that we don't compare two wrong results in the tests below.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contac

[GitHub] flink pull request #2976: [FLINK-5303] [table] Support for SQL GROUPING SETS...

2017-01-06 Thread twalthr
Github user twalthr commented on a diff in the pull request:

https://github.com/apache/flink/pull/2976#discussion_r94962145
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetAggregateRule.scala
 ---
@@ -50,30 +48,48 @@ class DataSetAggregateRule
   throw TableException("DISTINCT aggregates are currently not 
supported.")
 }
 
-// check if we have grouping sets
-val groupSets = agg.getGroupSets.size() != 1 || 
agg.getGroupSets.get(0) != agg.getGroupSet
-if (groupSets || agg.indicator) {
-  throw TableException("GROUPING SETS are currently not supported.")
-}
-
-!distinctAggs && !groupSets && !agg.indicator
+!distinctAggs
   }
 
   override def convert(rel: RelNode): RelNode = {
 val agg: LogicalAggregate = rel.asInstanceOf[LogicalAggregate]
 val traitSet: RelTraitSet = 
rel.getTraitSet.replace(DataSetConvention.INSTANCE)
 val convInput: RelNode = RelOptRule.convert(agg.getInput, 
DataSetConvention.INSTANCE)
 
-new DataSetAggregate(
-  rel.getCluster,
-  traitSet,
-  convInput,
-  agg.getNamedAggCalls,
-  rel.getRowType,
-  agg.getInput.getRowType,
-  agg.getGroupSet.toArray)
+if (agg.indicator) {
+agg.groupSets.map(set =>
+  new DataSetAggregate(
+rel.getCluster,
+traitSet,
+convInput,
+agg.getNamedAggCalls,
+rel.getRowType,
+agg.getInput.getRowType,
+set.toArray
+  ).asInstanceOf[RelNode]
+).reduce(
+  (rel1, rel2) => {
+new DataSetUnion(
+  rel.getCluster,
+  traitSet,
+  rel1, rel2,
--- End diff --

Missing line break.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #2976: [FLINK-5303] [table] Support for SQL GROUPING SETS...

2017-01-06 Thread twalthr
Github user twalthr commented on a diff in the pull request:

https://github.com/apache/flink/pull/2976#discussion_r94963463
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateReduceGroupFunction.scala
 ---
@@ -20,26 +20,29 @@ package org.apache.flink.table.runtime.aggregate
 import java.lang.Iterable
 
 import org.apache.flink.api.common.functions.RichGroupReduceFunction
-import org.apache.flink.types.Row
 import org.apache.flink.configuration.Configuration
+import org.apache.flink.types.Row
 import org.apache.flink.util.{Collector, Preconditions}
 
 import scala.collection.JavaConversions._
 
 /**
- * It wraps the aggregate logic inside of 
+ * It wraps the aggregate logic inside of
  * [[org.apache.flink.api.java.operators.GroupReduceOperator]].
  *
- * @param aggregates   The aggregate functions.
- * @param groupKeysMapping The index mapping of group keys between 
intermediate aggregate Row 
- * and output Row.
- * @param aggregateMapping The index mapping between aggregate function 
list and aggregated value
- * index in output Row.
+ * @param aggregates  The aggregate functions.
+ * @param groupKeysMappingThe index mapping of group keys between 
intermediate aggregate Row
+ *and output Row.
+ * @param aggregateMappingThe index mapping between aggregate function 
list and aggregated value
+ *index in output Row.
+ * @param groupingSetsMapping The index mapping of keys in grouping sets 
between intermediate
--- End diff --

It is not documented that this parameter can be null. I would recommend to 
use an `Option` here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #2976: [FLINK-5303] [table] Support for SQL GROUPING SETS...

2017-01-06 Thread twalthr
Github user twalthr commented on a diff in the pull request:

https://github.com/apache/flink/pull/2976#discussion_r94966326
  
--- Diff: 
flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/sql/GroupingSetsTest.java
 ---
@@ -0,0 +1,188 @@
+/*
+ * 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.api.java.batch.sql;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.operators.MapOperator;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.java.BatchTableEnvironment;
+import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
+import org.apache.flink.test.util.TestBaseUtils;
+import org.apache.flink.types.Row;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Comparator;
+import java.util.List;
+
+public class GroupingSetsTest {
--- End diff --

Please use the `TableProgramsTestBase` to reduce the build time.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #2976: [FLINK-5303] [table] Support for SQL GROUPING SETS...

2017-01-06 Thread twalthr
Github user twalthr commented on a diff in the pull request:

https://github.com/apache/flink/pull/2976#discussion_r94965456
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
 ---
@@ -349,6 +354,19 @@ object AggregateUtil {
 (groupingOffsetMapping, aggOffsetMapping)
   }
 
+  private def getGroupingSetsMapping(outputType: RelDataType): Array[(Int, 
Int)] = {
+val fields = outputType.getFieldList
+var mappingsBuffer = ArrayBuffer[(Int, Int)]()
+for (i <- fields.indices) {
+  for (j <- fields.indices) {
+if (fields(j).getName.equals("i$" + fields(i).getName)) {
--- End diff --

I know that it is unlikely but this could result in undesired behavior if 
the table contains `i$test` and `test`. Isn't there a nicer way to determine 
the grouping set mappings.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #2976: [FLINK-5303] [table] Support for SQL GROUPING SETS...

2017-01-06 Thread twalthr
Github user twalthr commented on a diff in the pull request:

https://github.com/apache/flink/pull/2976#discussion_r94964350
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateReduceGroupFunction.scala
 ---
@@ -87,6 +90,17 @@ class AggregateReduceGroupFunction(
 output.setField(after, 
aggregates(previous).evaluate(aggregateBuffer))
 }
 
+// Evaluate grouping sets additional values
+if (groupingSetsMapping != null && groupingSetsMapping.nonEmpty) {
+
+  val groupingFields = groupKeysMapping.map(_._1)
+  groupingSetsMapping.map {
--- End diff --

I would simplify these lines to one map instead of a map followed by 
foreach.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #2976: [FLINK-5303] [table] Support for SQL GROUPING SETS...

2017-01-06 Thread twalthr
Github user twalthr commented on a diff in the pull request:

https://github.com/apache/flink/pull/2976#discussion_r94960996
  
--- Diff: docs/dev/table_api.md ---
@@ -3744,6 +3745,50 @@ MIN(value)
 
   
 
+  Grouping functions
+  Description
+
+  
+
+  
+
+  
+{% highlight text %}
+GROUP_ID()
+{% endhighlight %}
+  
+  
+Returns an integer that uniquely identifies the combination of 
grouping keys.
+  
+
+
+
+  
+{% highlight text %}
+GROUPING(value)
--- End diff --

rename `value` to `expression` to be consistent with description.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #2976: [FLINK-5303] [table] Support for SQL GROUPING SETS...

2017-01-06 Thread twalthr
Github user twalthr commented on a diff in the pull request:

https://github.com/apache/flink/pull/2976#discussion_r94960370
  
--- Diff: docs/dev/table_api.md ---
@@ -1442,7 +1443,7 @@ Although not every SQL feature is implemented yet, 
some string combinations are
 
 {% highlight sql %}
 
-A, ABS, ABSOLUTE, ACTION, ADA, ADD, ADMIN, AFTER, ALL, ALLOCATE, ALLOW, 
ALTER, ALWAYS, AND, ANY, ARE, ARRAY, AS, ASC, ASENSITIVE, ASSERTION, 
ASSIGNMENT, ASYMMETRIC, AT, ATOMIC, ATTRIBUTE, ATTRIBUTES, AUTHORIZATION, AVG, 
BEFORE, BEGIN, BERNOULLI, BETWEEN, BIGINT, BINARY, BIT, BLOB, BOOLEAN, BOTH, 
BREADTH, BY, C, CALL, CALLED, CARDINALITY, CASCADE, CASCADED, CASE, CAST, 
CATALOG, CATALOG_NAME, CEIL, CEILING, CENTURY, CHAIN, CHAR, CHARACTER, 
CHARACTERISTICTS, CHARACTERS, CHARACTER_LENGTH, CHARACTER_SET_CATALOG, 
CHARACTER_SET_NAME, CHARACTER_SET_SCHEMA, CHAR_LENGTH, CHECK, CLASS_ORIGIN, 
CLOB, CLOSE, COALESCE, COBOL, COLLATE, COLLATION, COLLATION_CATALOG, 
COLLATION_NAME, COLLATION_SCHEMA, COLLECT, COLUMN, COLUMN_NAME, 
COMMAND_FUNCTION, COMMAND_FUNCTION_CODE, COMMIT, COMMITTED, CONDITION, 
CONDITION_NUMBER, CONNECT, CONNECTION, CONNECTION_NAME, CONSTRAINT, 
CONSTRAINTS, CONSTRAINT_CATALOG, CONSTRAINT_NAME, CONSTRAINT_SCHEMA, 
CONSTRUCTOR, CONTAINS, CONTINUE, CONVERT, CORR, CORRESPONDING, 
 COUNT, COVAR_POP, COVAR_SAMP, CREATE, CROSS, CUBE, CUME_DIST, CURRENT, 
CURRENT_CATALOG, CURRENT_DATE, CURRENT_DEFAULT_TRANSFORM_GROUP, CURRENT_PATH, 
CURRENT_ROLE, CURRENT_SCHEMA, CURRENT_TIME, CURRENT_TIMESTAMP, 
CURRENT_TRANSFORM_GROUP_FOR_TYPE, CURRENT_USER, CURSOR, CURSOR_NAME, CYCLE, 
DATA, DATABASE, DATE, DATETIME_INTERVAL_CODE, DATETIME_INTERVAL_PRECISION, DAY, 
DEALLOCATE, DEC, DECADE, DECIMAL, DECLARE, DEFAULT, DEFAULTS, DEFERRABLE, 
DEFERRED, DEFINED, DEFINER, DEGREE, DELETE, DENSE_RANK, DEPTH, DEREF, DERIVED, 
DESC, DESCRIBE, DESCRIPTION, DESCRIPTOR, DETERMINISTIC, DIAGNOSTICS, DISALLOW, 
DISCONNECT, DISPATCH, DISTINCT, DOMAIN, DOUBLE, DOW, DOY, DROP, DYNAMIC, 
DYNAMIC_FUNCTION, DYNAMIC_FUNCTION_CODE, EACH, ELEMENT, ELSE, END, END-EXEC, 
EPOCH, EQUALS, ESCAPE, EVERY, EXCEPT, EXCEPTION, EXCLUDE, EXCLUDING, EXEC, 
EXECUTE, EXISTS, EXP, EXPLAIN, EXTEND, EXTERNAL, EXTRACT, FALSE, FETCH, FILTER, 
FINAL, FIRST, FIRST_VALUE, FLOAT, FLOOR, FOLLOWING, FOR, FOREIGN, FORTRAN, 
FOUND, FRAC_SECON
 D, FREE, FROM, FULL, FUNCTION, FUSION, G, GENERAL, GENERATED, GET, GLOBAL, GO, 
GOTO, GRANT, GRANTED, GROUP, GROUPING, HAVING, HIERARCHY, HOLD, HOUR, IDENTITY, 
IMMEDIATE, IMPLEMENTATION, IMPORT, IN, INCLUDING, INCREMENT, INDICATOR, 
INITIALLY, INNER, INOUT, INPUT, INSENSITIVE, INSERT, INSTANCE, INSTANTIABLE, 
INT, INTEGER, INTERSECT, INTERSECTION, INTERVAL, INTO, INVOKER, IS, ISOLATION, 
JAVA, JOIN, K, KEY, KEY_MEMBER, KEY_TYPE, LABEL, LANGUAGE, LARGE, LAST, 
LAST_VALUE, LATERAL, LEADING, LEFT, LENGTH, LEVEL, LIBRARY, LIKE, LIMIT, LN, 
LOCAL, LOCALTIME, LOCALTIMESTAMP, LOCATOR, LOWER, M, MAP, MATCH, MATCHED, MAX, 
MAXVALUE, MEMBER, MERGE, MESSAGE_LENGTH, MESSAGE_OCTET_LENGTH, MESSAGE_TEXT, 
METHOD, MICROSECOND, MILLENNIUM, MIN, MINUTE, MINVALUE, MOD, MODIFIES, MODULE, 
MONTH, MORE, MULTISET, MUMPS, NAME, NAMES, NATIONAL, NATURAL, NCHAR, NCLOB, 
NESTING, NEW, NEXT, NO, NONE, NORMALIZE, NORMALIZED, NOT, NULL, NULLABLE, 
NULLIF, NULLS, NUMBER, NUMERIC, OBJECT, OCTETS, OCTET_LENGTH, OF, OFFSET, OL
 D, ON, ONLY, OPEN, OPTION, OPTIONS, OR, ORDER, ORDERING, ORDINALITY, OTHERS, 
OUT, OUTER, OUTPUT, OVER, OVERLAPS, OVERLAY, OVERRIDING, PAD, PARAMETER, 
PARAMETER_MODE, PARAMETER_NAME, PARAMETER_ORDINAL_POSITION, 
PARAMETER_SPECIFIC_CATALOG, PARAMETER_SPECIFIC_NAME, PARAMETER_SPECIFIC_SCHEMA, 
PARTIAL, PARTITION, PASCAL, PASSTHROUGH, PATH, PERCENTILE_CONT, 
PERCENTILE_DISC, PERCENT_RANK, PLACING, PLAN, PLI, POSITION, POWER, PRECEDING, 
PRECISION, PREPARE, PRESERVE, PRIMARY, PRIOR, PRIVILEGES, PROCEDURE, PUBLIC, 
QUARTER, RANGE, RANK, READ, READS, REAL, RECURSIVE, REF, REFERENCES, 
REFERENCING, REGR_AVGX, REGR_AVGY, REGR_COUNT, REGR_INTERCEPT, REGR_R2, 
REGR_SLOPE, REGR_SXX, REGR_SXY, REGR_SYY, RELATIVE, RELEASE, REPEATABLE, RESET, 
RESTART, RESTRICT, RESULT, RETURN, RETURNED_CARDINALITY, RETURNED_LENGTH, 
RETURNED_OCTET_LENGTH, RETURNED_SQLSTATE, RETURNS, REVOKE, RIGHT, ROLE, 
ROLLBACK, ROLLUP, ROUTINE, ROUTINE_CATALOG, ROUTINE_NAME, ROUTINE_SCHEMA, ROW, 
ROWS, ROW_COUNT, ROW_NUMBER, SAVEPOINT, S
 CALE, SCHEMA, SCHEMA_NAME, SCOPE, SCOPE_CATALOGS, SCOPE_NAME, SCOPE_SCHEMA, 
SCROLL, SEARCH, SECOND, SECTION, SECURITY, SELECT, SELF, SENSITIVE, SEQUENCE, 
SERIALIZABLE, SERVER, SERVER_NAME, SESSION, SESSION_USER, SET, SETS, SIMILAR, 
SIMPLE, SIZE, SMALLINT, SOME, SOURCE, SPACE, SPECIFIC, SPECIFICTYPE, 
SPECIFIC_NAME, SQL, SQLEXCEPTION, SQLSTATE, SQLWARNING, SQL_TSI_DAY, 
SQL_TSI_FRAC_SECOND, SQL_TSI_HOUR, SQL_TSI_MICROSECOND, SQL_TSI_MINUTE, 
SQL_TSI_MONTH, SQL_TSI_QUARTER, SQL_TSI_SECOND, SQL_TSI_WEEK, SQL_TSI_YEAR, 
SQRT, START, STATE, STATEMENT, STATIC, STDDEV_POP, STDDEV_SAMP, STREAM, 
STRUCTURE, STYLE, SUBCLASS_ORIGIN, SU

[GitHub] flink pull request #2976: [FLINK-5303] [table] Support for SQL GROUPING SETS...

2017-01-06 Thread twalthr
Github user twalthr commented on a diff in the pull request:

https://github.com/apache/flink/pull/2976#discussion_r94961010
  
--- Diff: docs/dev/table_api.md ---
@@ -3744,6 +3745,50 @@ MIN(value)
 
   
 
+  Grouping functions
+  Description
+
+  
+
+  
+
+  
+{% highlight text %}
+GROUP_ID()
+{% endhighlight %}
+  
+  
+Returns an integer that uniquely identifies the combination of 
grouping keys.
+  
+
+
+
+  
+{% highlight text %}
+GROUPING(value)
+{% endhighlight %}
+  
+  
+Returns 1 if expression is rolled up in the current row’s 
grouping set, 0 otherwise.
+  
+
+
+
+  
+{% highlight text %}
+GROUPING_ID(value [, value]* )
--- End diff --

rename `value` to `expression` to be consistent with description.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #2976: [FLINK-5303] [table] Support for SQL GROUPING SETS...

2017-01-16 Thread asfgit
Github user asfgit closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---