You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by se...@apache.org on 2020/03/02 14:29:55 UTC
[flink] branch master updated (8cdef52 -> 2806215)
This is an automated email from the ASF dual-hosted git repository.
sewen pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git.
from 8cdef52 [FLINK-15633][build] Bump javadoc-plugin to 3.1.1
new cc75e01 [FLINK-2336][DataStream API] Fix ArrayIndexOufOBoundsException in TypeExtractor for type erased lambdas
new d62eb7e [hotfix][docs] Fix the mismatch between java and scala CEP examples
new 2806215 [hotfix][table] Remove unused import in DataStreamConversions
The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails. The revisions
listed as "add" were already present in the repository and have only
been added to this reference.
Summary of changes:
docs/dev/libs/cep.md | 3 +++
.../java/org/apache/flink/api/java/typeutils/TypeExtractor.java | 5 +++++
.../org/apache/flink/api/java/typeutils/LambdaExtractionTest.java | 6 ++++++
.../org/apache/flink/table/planner/DataStreamConversions.scala | 1 -
4 files changed, 14 insertions(+), 1 deletion(-)
[flink] 01/03: [FLINK-2336][DataStream API] Fix
ArrayIndexOufOBoundsException in TypeExtractor for type erased lambdas
Posted by se...@apache.org.
This is an automated email from the ASF dual-hosted git repository.
sewen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
commit cc75e01abcb6b6eed50591c94db9e2a26f179518
Author: GuoWei Ma <gu...@gmail.com>
AuthorDate: Thu Feb 27 16:12:27 2020 +0800
[FLINK-2336][DataStream API] Fix ArrayIndexOufOBoundsException in TypeExtractor for type erased lambdas
This closes #11234
---
.../java/org/apache/flink/api/java/typeutils/TypeExtractor.java | 5 +++++
.../org/apache/flink/api/java/typeutils/LambdaExtractionTest.java | 6 ++++++
2 files changed, 11 insertions(+)
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
index 81cd4d2..acb1ec9 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
@@ -953,6 +953,11 @@ public class TypeExtractor {
break;
}
}
+
+ if (inputTypeHierarchy.size() == 0) {
+ return null;
+ }
+
ParameterizedType baseClass = (ParameterizedType) inputTypeHierarchy.get(inputTypeHierarchy.size() - 1);
TypeInformation<?> info = null;
diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/LambdaExtractionTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/LambdaExtractionTest.java
index 1d5cf22..36d3637 100644
--- a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/LambdaExtractionTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/LambdaExtractionTest.java
@@ -232,6 +232,12 @@ public class LambdaExtractionTest {
}
@Test
+ public void testLambdaWithoutTypeErasure() {
+ TypeInformation<?> ti = TypeExtractor.getMapReturnTypes(Tuple1::of, BasicTypeInfo.STRING_TYPE_INFO, null, true);
+ assertTrue(ti instanceof MissingTypeInfo);
+ }
+
+ @Test
public void testPartitionerLambda() {
Partitioner<Tuple2<Integer, String>> partitioner = (key, numPartitions) -> key.f1.length() % numPartitions;
final TypeInformation<?> ti = TypeExtractor.getPartitionerTypes(partitioner, null, true);
[flink] 02/03: [hotfix][docs] Fix the mismatch between java and
scala CEP examples
Posted by se...@apache.org.
This is an automated email from the ASF dual-hosted git repository.
sewen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
commit d62eb7ed08d277b07f67c483987a30a51a7a2a59
Author: shuai-xu <sh...@foxmail.com>
AuthorDate: Mon Mar 2 15:08:50 2020 +0800
[hotfix][docs] Fix the mismatch between java and scala CEP examples
This closes #11278
---
docs/dev/libs/cep.md | 3 +++
1 file changed, 3 insertions(+)
diff --git a/docs/dev/libs/cep.md b/docs/dev/libs/cep.md
index a6b9251..8e196b6 100644
--- a/docs/dev/libs/cep.md
+++ b/docs/dev/libs/cep.md
@@ -206,6 +206,9 @@ start.timesOrMore(2);
// expecting 2 or more occurrences and repeating as many as possible
start.timesOrMore(2).greedy();
+// expecting 0, 2 or more occurrences
+start.timesOrMore(2).optional()
+
// expecting 0, 2 or more occurrences and repeating as many as possible
start.timesOrMore(2).optional().greedy();
{% endhighlight %}
[flink] 03/03: [hotfix][table] Remove unused import in
DataStreamConversions
Posted by se...@apache.org.
This is an automated email from the ASF dual-hosted git repository.
sewen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
commit 2806215c0c4d88c336ee57ab639d24685b485273
Author: Paul Lin <pa...@gmail.com>
AuthorDate: Sun Mar 1 18:36:13 2020 +0800
[hotfix][table] Remove unused import in DataStreamConversions
This closes #11271
---
.../scala/org/apache/flink/table/planner/DataStreamConversions.scala | 1 -
1 file changed, 1 deletion(-)
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/DataStreamConversions.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/DataStreamConversions.scala
index e4cd58b..199a32e 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/DataStreamConversions.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/DataStreamConversions.scala
@@ -20,7 +20,6 @@ package org.apache.flink.table.planner
import java.lang.{Boolean => JBool}
-import org.apache.calcite.rel.RelNode
import org.apache.flink.api.common.functions.MapFunction
import org.apache.flink.api.common.typeinfo.{SqlTimeTypeInfo, TypeInformation}
import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}