[ 
https://issues.apache.org/jira/browse/FLINK-629?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14064828#comment-14064828
 ] 

Gyula Fora commented on FLINK-629:
----------------------------------

As for Stephan's comment on null field serialization, I think we should create 
another issue for that.

We have modified the TupleSerializers so now it can work on null objects, but 
some tests fail and it probably needs some workn (we suspect it is something 
around byte skipping).

A basic solution could be something like this:
{code}
@Override
        public void serialize(T value, DataOutputView target) throws 
IOException {
                for (int i = 0; i < arity; i++) {

                        try {
                                Object o = value.getFieldNotNull(i);
                                // Indicate that the field is not null
                                target.writeBoolean(false);
                                fieldSerializers[i].serialize(o, target);
                        } catch (NullFieldException npex) {
                                //If null don't serialize anything
                                target.writeBoolean(true);
                        }

                }
        }

        @Override
        public T deserialize(T reuse, DataInputView source) throws IOException {
                for (int i = 0; i < arity; i++) {
                        Object field;
                        //Only deserialize if field was not null
                        boolean isNull = source.readBoolean();
                        if (isNull) {
                                field = null;
                        } else {
                                field = 
fieldSerializers[i].deserialize(reuse.getField(i), source);
                        }
                        reuse.setField(field, i);
                }
                return reuse;
        }
{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)

Reply via email to