[
https://issues.apache.org/jira/browse/FLINK-629?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14064722#comment-14064722
]
Stephan Ewen commented on FLINK-629:
------------------------------------
My first intuition is to have the primitive type serializers NOT handle null
values, but make that part of the Tuple / Pojo Serializer.
That way we retain a way of having efficien non-nullable types. Object types
are nullable by default, primitive types never are.
For example:
A {{Tuple2<Long, Long>}} would be represented as
{{<nullFlag><longValue><nullFlag><longValue>}}.
The type below could be represented as {{long, long}}, which is more efficient.
{code}
public class Edge {
public long v1;
public long v2;
}
{code}
> Add support for null values to the java api
> -------------------------------------------
>
> Key: FLINK-629
> URL: https://issues.apache.org/jira/browse/FLINK-629
> Project: Flink
> Issue Type: Improvement
> Components: Java API
> Reporter: Stephan Ewen
> Assignee: Gyula Fora
> Priority: Critical
> Labels: github-import
> Fix For: pre-apache
>
>
> Currently, many runtime operations fail when encountering a null value. Tuple
> serialization should allow null fields.
> I suggest to add a method to the tuples called `getFieldNotNull()` which
> throws a meaningful exception when the accessed field is null. That way, we
> simplify the logic of operators that should not dead with null fields, like
> key grouping or aggregations.
> Even though SQL allows grouping and aggregating of null values, I suggest to
> exclude this from the java api, because the SQL semantics of aggregating null
> fields are messy.
> ---------------- Imported from GitHub ----------------
> Url: https://github.com/stratosphere/stratosphere/issues/629
> Created by: [StephanEwen|https://github.com/StephanEwen]
> Labels: enhancement, java api,
> Milestone: Release 0.5.1
> Created at: Wed Mar 26 00:27:49 CET 2014
> State: open
--
This message was sent by Atlassian JIRA
(v6.2#6252)