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

ASF GitHub Bot commented on FLINK-3702:
---------------------------------------

Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2094#discussion_r84765869
  
    --- Diff: 
flink-core/src/main/java/org/apache/flink/api/java/typeutils/FieldAccessor.java 
---
    @@ -0,0 +1,324 @@
    +/*
    + * 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.api.java.typeutils;
    +
    +import org.apache.flink.annotation.PublicEvolving;
    +import org.apache.flink.api.common.operators.Keys;
    +import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
    +import org.apache.flink.api.common.typeinfo.InvalidFieldReferenceException;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.api.java.typeutils.runtime.FieldSerializer;
    +
    +import java.io.IOException;
    +import java.io.ObjectInputStream;
    +import java.io.ObjectOutputStream;
    +import java.io.Serializable;
    +import java.lang.reflect.Array;
    +import java.lang.reflect.Field;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +import static org.apache.flink.util.Preconditions.checkNotNull;
    +
    +
    +/**
    + * These classes encapsulate the logic of accessing a field specified by 
the user as either an index
    + * or a field expression string. TypeInformation can also be requested for 
the field.
    + * The position index might specify a field of a Tuple, an array, or a 
simple type (only "0th field").
    + *
    + * Field expressions that specify nested fields (e.g. "f1.a.foo") result 
in nested field accessors.
    + * These penetrate one layer, and then delegate the rest of the work to an 
"innerAccesor".
    + * (see PojoFieldAccessor, RecursiveTupleFieldAccessor, 
ProductFieldAccessor)
    + */
    +@PublicEvolving
    --- End diff --
    
    `FieldAccessor` was `@Internal` before. 
    Why changing it to `@PublicEvolving`?


> DataStream API PojoFieldAccessor doesn't support nested POJOs
> -------------------------------------------------------------
>
>                 Key: FLINK-3702
>                 URL: https://issues.apache.org/jira/browse/FLINK-3702
>             Project: Flink
>          Issue Type: Improvement
>          Components: DataStream API
>    Affects Versions: 1.0.0
>            Reporter: Robert Metzger
>            Assignee: Gabor Gevay
>
> The {{PojoFieldAccessor}} (which is used by {{.sum(String)}} and similar 
> methods) doesn't support nested POJOs right now.
> As part of FLINK-3697 I'll add a check for a nested POJO and fail with an 
> exception.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to