Stephan Ewen created FLINK-1145:
-----------------------------------
Summary: POJO Type extractor bug with type variables
Key: FLINK-1145
URL: https://issues.apache.org/jira/browse/FLINK-1145
Project: Flink
Issue Type: Bug
Reporter: Stephan Ewen
The following program incorrectly states that there are duplicate
getters/setters.
{code}
public static class Vertex<K, V> {
private K key1;
private K key2;
private V value;
public Vertex() {}
public Vertex(K key, V value) {
this.key1 = key;
this.key2 = key;
this.value = value;
}
public Vertex(K key1, K key2, V value) {
this.key1 = key1;
this.key2 = key2;
this.value = value;
}
public void setKey1(K key1) {
this.key1 = key1;
}
public void setKey2(K key2) {
this.key2 = key2;
}
public K getKey1() {
return key1;
}
public K getKey2() {
return key2;
}
public void setValue(V value) {
this.value = value;
}
public V getValue() {
return value;
}
}
public static void main(String[] args) throws Exception {
ExecutionEnvironment env =
ExecutionEnvironment.getExecutionEnvironment();
DataSet<Vertex<Long, Double>> set = env.fromElements(new
Vertex<Long, Double>(0L, 3.0), new Vertex<Long, Double>(1L, 1.0));
set.print();
env.execute();
}
{code}
The exception is
{code}
Exception in thread "main" java.lang.IllegalStateException: Detected more than
one getters
at
org.apache.flink.api.java.typeutils.TypeExtractor.isValidPojoField(TypeExtractor.java:981)
at
org.apache.flink.api.java.typeutils.TypeExtractor.analyzePojo(TypeExtractor.java:1025)
at
org.apache.flink.api.java.typeutils.TypeExtractor.privateGetForClass(TypeExtractor.java:937)
at
org.apache.flink.api.java.typeutils.TypeExtractor.privateGetForClass(TypeExtractor.java:863)
at
org.apache.flink.api.java.typeutils.TypeExtractor.privateGetForObject(TypeExtractor.java:1146)
at
org.apache.flink.api.java.typeutils.TypeExtractor.getForObject(TypeExtractor.java:1116)
at
org.apache.flink.api.java.ExecutionEnvironment.fromElements(ExecutionEnvironment.java:466)
at test.Test.main(Test.java:74)
{code}
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)