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

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

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

    https://github.com/apache/flink/pull/2094#discussion_r76576801
  
    --- Diff: 
flink-core/src/test/java/org/apache/flink/api/java/typeutils/FieldAccessorTest.java
 ---
    @@ -0,0 +1,336 @@
    +/*
    + * 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 static org.junit.Assert.*;
    +
    +import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
    +import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
    +import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.common.typeinfo.InvalidFieldReferenceException;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.api.java.tuple.Tuple3;
    +import org.junit.Test;
    +
    +public class FieldAccessorTest {
    +
    +   // Note, that AggregationFunctionTest indirectly also tests 
FieldAccessors.
    +   // ProductFieldAccessor is tested in CaseClassTypeInfoTest.
    +
    +   @Test
    +   public void testFlatTuple() {
    +           Tuple2<String, Integer> t = Tuple2.of("aa", 5);
    +           TupleTypeInfo<Tuple2<String, Integer>> tpeInfo =
    +                           (TupleTypeInfo<Tuple2<String, Integer>>) 
TypeExtractor.getForObject(t);
    +
    +           FieldAccessor<Tuple2<String, Integer>, String> f0 = 
tpeInfo.getFieldAccessor("f0", null);
    +           assertEquals("aa", f0.get(t));
    +           assertEquals("aa", t.f0);
    +           t = f0.set(t, "b");
    +           assertEquals("b", f0.get(t));
    +           assertEquals("b", t.f0);
    +
    +           FieldAccessor<Tuple2<String, Integer>, Integer> f1 = 
tpeInfo.getFieldAccessor("f1", null);
    +           assertEquals(5, (int) f1.get(t));
    +           assertEquals(5, (int) t.f1);
    +           t = f1.set(t, 7);
    +           assertEquals(7, (int) f1.get(t));
    +           assertEquals(7, (int) t.f1);
    +           assertEquals("b", f0.get(t));
    +           assertEquals("b", t.f0);
    +
    +
    +           FieldAccessor<Tuple2<String, Integer>, Integer> f1n = 
tpeInfo.getFieldAccessor(1, null);
    +           assertEquals(7, (int) f1n.get(t));
    +           assertEquals(7, (int) t.f1);
    +           t = f1n.set(t, 10);
    +           assertEquals(10, (int) f1n.get(t));
    +           assertEquals(10, (int) f1.get(t));
    +           assertEquals(10, (int) t.f1);
    +           assertEquals("b", f0.get(t));
    +           assertEquals("b", t.f0);
    +
    +           // This is technically valid (the ".0" is selecting the 0th 
field of a basic type).
    +           FieldAccessor<Tuple2<String, Integer>, String> f0_0 = 
tpeInfo.getFieldAccessor("f0.0", null);
    +           assertEquals("b", f0_0.get(t));
    +           assertEquals("b", t.f0);
    +           t = f0_0.set(t, "cc");
    +           assertEquals("cc", f0_0.get(t));
    +           assertEquals("cc", t.f0);
    +
    +           try {
    +                   FieldAccessor<Tuple2<String, Integer>, String> bad = 
tpeInfo.getFieldAccessor("almafa", null);
    +                   assertFalse("Expected exception, because of bad field 
name", false);
    +           } catch (InvalidFieldReferenceException ex) {
    +                   // OK
    +           }
    +   }
    +
    +   @Test
    +   public void testTupleInTuple() {
    +           Tuple2<String, Tuple3<Integer, Long, Double>> t = 
Tuple2.of("aa", Tuple3.of(5, 9L, 2.0));
    +           TupleTypeInfo<Tuple2<String, Tuple3<Integer, Long, Double>>> 
tpeInfo =
    +                           (TupleTypeInfo<Tuple2<String, Tuple3<Integer, 
Long, Double>>>)TypeExtractor.getForObject(t);
    +
    +           FieldAccessor<Tuple2<String, Tuple3<Integer, Long, Double>>, 
String> f0 = tpeInfo.getFieldAccessor("f0", null);
    +           assertEquals("aa", f0.get(t));
    +           assertEquals("aa", t.f0);
    +
    +           FieldAccessor<Tuple2<String, Tuple3<Integer, Long, Double>>, 
Double> f1f2 = tpeInfo.getFieldAccessor("f1.f2", null);
    +           assertEquals(2.0, f1f2.get(t), 0);
    +           assertEquals(2.0, t.f1.f2, 0);
    +           t = f1f2.set(t, 3.0);
    +           assertEquals(3.0, f1f2.get(t), 0);
    +           assertEquals(3.0, t.f1.f2, 0);
    +           assertEquals("aa", f0.get(t));
    +           assertEquals("aa", t.f0);
    +
    +           FieldAccessor<Tuple2<String, Tuple3<Integer, Long, Double>>, 
Tuple3<Integer, Long, Double>> f1 = tpeInfo.getFieldAccessor("f1", null);
    +           assertEquals(Tuple3.of(5, 9L, 3.0), f1.get(t));
    +           assertEquals(Tuple3.of(5, 9L, 3.0), t.f1);
    +           t = f1.set(t, Tuple3.of(8, 12L, 4.0));
    +           assertEquals(Tuple3.of(8, 12L, 4.0), f1.get(t));
    +           assertEquals(Tuple3.of(8, 12L, 4.0), t.f1);
    +           assertEquals("aa", f0.get(t));
    +           assertEquals("aa", t.f0);
    +
    +           FieldAccessor<Tuple2<String, Tuple3<Integer, Long, Double>>, 
Tuple3<Integer, Long, Double>> f1n = tpeInfo.getFieldAccessor(1, null);
    +           assertEquals(Tuple3.of(8, 12L, 4.0), f1n.get(t));
    +           assertEquals(Tuple3.of(8, 12L, 4.0), t.f1);
    +           t = f1n.set(t, Tuple3.of(10, 13L, 5.0));
    +           assertEquals(Tuple3.of(10, 13L, 5.0), f1n.get(t));
    +           assertEquals(Tuple3.of(10, 13L, 5.0), f1.get(t));
    +           assertEquals(Tuple3.of(10, 13L, 5.0), t.f1);
    +           assertEquals("aa", f0.get(t));
    +           assertEquals("aa", t.f0);
    +   }
    +
    +   @Test
    +   @SuppressWarnings("unchecked")
    +   public void tupleFieldAccessorOutOfBoundsTest() {
    --- End diff --
    
    39e4d96e686d621ca6609a9f80665862c2a004d3


> 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