[ 
https://issues.apache.org/jira/browse/BEAM-7129?focusedWorklogId=230840&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-230840
 ]

ASF GitHub Bot logged work on BEAM-7129:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 22/Apr/19 19:01
            Start Date: 22/Apr/19 19:01
    Worklog Time Spent: 10m 
      Work Description: amaliujia commented on pull request #8375: [BEAM-7129] 
BeamEnumerableConverter does not handle null values for all types
URL: https://github.com/apache/beam/pull/8375#discussion_r277396492
 
 

 ##########
 File path: 
sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverterTest.java
 ##########
 @@ -57,137 +65,159 @@
   static PipelineOptions options = PipelineOptionsFactory.create();
   static RelOptCluster cluster = RelOptCluster.create(new VolcanoPlanner(), 
rexBuilder);
 
-  @Test
-  public void testToEnumerable_collectSingle() {
-    Schema schema = Schema.builder().addInt64Field("id").build();
-    RelDataType type = CalciteUtils.toCalciteRowType(schema, TYPE_FACTORY);
-    ImmutableList<ImmutableList<RexLiteral>> tuples =
-        
ImmutableList.of(ImmutableList.of(rexBuilder.makeBigintLiteral(BigDecimal.ZERO)));
-    BeamRelNode node = new BeamValuesRel(cluster, type, tuples, null);
-
-    Enumerable<Object> enumerable = 
BeamEnumerableConverter.toEnumerable(options, node);
-    Enumerator<Object> enumerator = enumerable.enumerator();
-
-    assertTrue(enumerator.moveNext());
-    assertEquals(0L, enumerator.current());
-    assertFalse(enumerator.moveNext());
-    enumerator.close();
-  }
-
-  @Test
-  public void testToEnumerable_collectMultiple() {
-    Schema schema = 
Schema.builder().addInt64Field("id").addInt64Field("otherid").build();
-    RelDataType type = CalciteUtils.toCalciteRowType(schema, TYPE_FACTORY);
-    ImmutableList<ImmutableList<RexLiteral>> tuples =
-        ImmutableList.of(
-            ImmutableList.of(
-                rexBuilder.makeBigintLiteral(BigDecimal.ZERO),
-                rexBuilder.makeBigintLiteral(BigDecimal.ONE)));
-    BeamRelNode node = new BeamValuesRel(cluster, type, tuples, null);
-
-    Enumerable<Object> enumerable = 
BeamEnumerableConverter.toEnumerable(options, node);
-    Enumerator<Object> enumerator = enumerable.enumerator();
-
-    assertTrue(enumerator.moveNext());
-    Object[] row = (Object[]) enumerator.current();
-    assertEquals(2, row.length);
-    assertEquals(0L, row[0]);
-    assertEquals(1L, row[1]);
-    assertFalse(enumerator.moveNext());
-    enumerator.close();
-  }
-
-  @Test
-  public void testToListRow_collectMultiple() {
-    Schema schema = 
Schema.builder().addInt64Field("id").addInt64Field("otherid").build();
-    RelDataType type = CalciteUtils.toCalciteRowType(schema, TYPE_FACTORY);
-    ImmutableList<ImmutableList<RexLiteral>> tuples =
-        ImmutableList.of(
-            ImmutableList.of(
-                rexBuilder.makeBigintLiteral(BigDecimal.ZERO),
-                rexBuilder.makeBigintLiteral(BigDecimal.ONE)));
-    BeamRelNode node = new BeamValuesRel(cluster, type, tuples, null);
-
-    List<Row> rowList = BeamEnumerableConverter.toRowList(options, node);
-    assertTrue(rowList.size() == 1);
-    assertEquals(Row.withSchema(schema).addValues(0L, 1L).build(), 
rowList.get(0));
-  }
-
-  @Test
-  public void testToEnumerable_collectNullValue() {
-    Schema schema = Schema.builder().addNullableField("id", 
FieldType.INT64).build();
-    RelDataType type = CalciteUtils.toCalciteRowType(schema, TYPE_FACTORY);
-    ImmutableList<ImmutableList<RexLiteral>> tuples =
-        ImmutableList.of(
-            ImmutableList.of(
-                rexBuilder.makeNullLiteral(
-                    CalciteUtils.toRelDataType(TYPE_FACTORY, 
FieldType.INT64))));
-    BeamRelNode node = new BeamValuesRel(cluster, type, tuples, null);
-
-    Enumerable<Object> enumerable = 
BeamEnumerableConverter.toEnumerable(options, node);
-    Enumerator<Object> enumerator = enumerable.enumerator();
-
-    assertTrue(enumerator.moveNext());
-    Object row = enumerator.current();
-    assertEquals(null, row);
-    assertFalse(enumerator.moveNext());
-    enumerator.close();
-  }
+  /** Non-parameterized tests for BeamEnumerableConverter. */
+  @RunWith(JUnit4.class)
+  public static class SpecificTypeTests {
+
+    @Test
+    public void testToEnumerable_collectSingle() {
+      Schema schema = Schema.builder().addInt64Field("id").build();
+      RelDataType type = CalciteUtils.toCalciteRowType(schema, TYPE_FACTORY);
+      ImmutableList<ImmutableList<RexLiteral>> tuples =
+          
ImmutableList.of(ImmutableList.of(rexBuilder.makeBigintLiteral(BigDecimal.ZERO)));
+      BeamRelNode node = new BeamValuesRel(cluster, type, tuples, null);
+
+      Enumerable<Object> enumerable = 
BeamEnumerableConverter.toEnumerable(options, node);
+      Enumerator<Object> enumerator = enumerable.enumerator();
+
+      assertTrue(enumerator.moveNext());
+      assertEquals(0L, enumerator.current());
+      assertFalse(enumerator.moveNext());
+      enumerator.close();
+    }
 
-  private static class FakeTable extends BaseBeamTable {
-    public FakeTable() {
-      super(null);
+    @Test
+    public void testToEnumerable_collectMultiple() {
+      Schema schema = 
Schema.builder().addInt64Field("id").addInt64Field("otherid").build();
+      RelDataType type = CalciteUtils.toCalciteRowType(schema, TYPE_FACTORY);
+      ImmutableList<ImmutableList<RexLiteral>> tuples =
+          ImmutableList.of(
+              ImmutableList.of(
+                  rexBuilder.makeBigintLiteral(BigDecimal.ZERO),
+                  rexBuilder.makeBigintLiteral(BigDecimal.ONE)));
+      BeamRelNode node = new BeamValuesRel(cluster, type, tuples, null);
+
+      Enumerable<Object> enumerable = 
BeamEnumerableConverter.toEnumerable(options, node);
+      Enumerator<Object> enumerator = enumerable.enumerator();
+
+      assertTrue(enumerator.moveNext());
+      Object[] row = (Object[]) enumerator.current();
+      assertEquals(2, row.length);
+      assertEquals(0L, row[0]);
+      assertEquals(1L, row[1]);
+      assertFalse(enumerator.moveNext());
+      enumerator.close();
     }
 
-    @Override
-    public PCollection.IsBounded isBounded() {
-      return null;
+    @Test
+    public void testToListRow_collectMultiple() {
+      Schema schema = 
Schema.builder().addInt64Field("id").addInt64Field("otherid").build();
+      RelDataType type = CalciteUtils.toCalciteRowType(schema, TYPE_FACTORY);
+      ImmutableList<ImmutableList<RexLiteral>> tuples =
+          ImmutableList.of(
+              ImmutableList.of(
+                  rexBuilder.makeBigintLiteral(BigDecimal.ZERO),
+                  rexBuilder.makeBigintLiteral(BigDecimal.ONE)));
+      BeamRelNode node = new BeamValuesRel(cluster, type, tuples, null);
+
+      List<Row> rowList = BeamEnumerableConverter.toRowList(options, node);
+      assertTrue(rowList.size() == 1);
+      assertEquals(Row.withSchema(schema).addValues(0L, 1L).build(), 
rowList.get(0));
     }
 
-    @Override
-    public PCollection<Row> buildIOReader(PBegin begin) {
-      return null;
+    private static class FakeTable extends BaseBeamTable {
+      public FakeTable() {
+        super(null);
+      }
+
+      @Override
+      public PCollection.IsBounded isBounded() {
+        return null;
+      }
+
+      @Override
+      public PCollection<Row> buildIOReader(PBegin begin) {
+        return null;
+      }
+
+      @Override
+      public POutput buildIOWriter(PCollection<Row> input) {
+        input.apply(
+            ParDo.of(
+                new DoFn<Row, Void>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext context) {}
+                }));
+        return PDone.in(input.getPipeline());
+      }
     }
 
-    @Override
-    public POutput buildIOWriter(PCollection<Row> input) {
-      input.apply(
-          ParDo.of(
-              new DoFn<Row, Void>() {
-                @ProcessElement
-                public void processElement(ProcessContext context) {}
-              }));
-      return PDone.in(input.getPipeline());
+    @Test
+    public void testToEnumerable_count() {
+      Schema schema = Schema.builder().addInt64Field("id").build();
+      RelDataType type = CalciteUtils.toCalciteRowType(schema, TYPE_FACTORY);
+      ImmutableList<ImmutableList<RexLiteral>> tuples =
+          ImmutableList.of(
+              ImmutableList.of(rexBuilder.makeBigintLiteral(BigDecimal.ZERO)),
+              ImmutableList.of(rexBuilder.makeBigintLiteral(BigDecimal.ONE)));
+      BeamRelNode node =
+          new BeamIOSinkRel(
+              cluster,
+              RelOptTableImpl.create(null, type, ImmutableList.of(), null),
+              null,
+              new BeamValuesRel(cluster, type, tuples, null),
+              null,
+              null,
+              null,
+              false,
+              new FakeTable(),
+              null);
+
+      Enumerable<Object> enumerable = 
BeamEnumerableConverter.toEnumerable(options, node);
+      Enumerator<Object> enumerator = enumerable.enumerator();
+
+      assertTrue(enumerator.moveNext());
+      assertEquals(2L, enumerator.current());
+      assertFalse(enumerator.moveNext());
+      enumerator.close();
     }
   }
 
-  @Test
-  public void testToEnumerable_count() {
-    Schema schema = Schema.builder().addInt64Field("id").build();
-    RelDataType type = CalciteUtils.toCalciteRowType(schema, TYPE_FACTORY);
-    ImmutableList<ImmutableList<RexLiteral>> tuples =
-        ImmutableList.of(
-            ImmutableList.of(rexBuilder.makeBigintLiteral(BigDecimal.ZERO)),
-            ImmutableList.of(rexBuilder.makeBigintLiteral(BigDecimal.ONE)));
-    BeamRelNode node =
-        new BeamIOSinkRel(
-            cluster,
-            RelOptTableImpl.create(null, type, ImmutableList.of(), null),
-            null,
-            new BeamValuesRel(cluster, type, tuples, null),
-            null,
-            null,
-            null,
-            false,
-            new FakeTable(),
-            null);
-
-    Enumerable<Object> enumerable = 
BeamEnumerableConverter.toEnumerable(options, node);
-    Enumerator<Object> enumerator = enumerable.enumerator();
-
-    assertTrue(enumerator.moveNext());
-    assertEquals(2L, enumerator.current());
-    assertFalse(enumerator.moveNext());
-    enumerator.close();
+  /**
+   * Tests for BeamEnumerableConverter parameterized by field type. Only 
primitive field types are
+   * included.
+   */
+  @RunWith(Parameterized.class)
+  public static class PrimitiveTypeTests {
+    @Parameterized.Parameter public FieldType fieldType;
+
+    @Parameterized.Parameters(name = "{0}")
+    public static Iterable<Object[]> data() {
+      return Arrays.stream(TypeName.values())
+          .filter(TypeName::isPrimitiveType)
+          .map(typeName -> new Object[] {FieldType.of(typeName)})
 
 Review comment:
   Could we also mock logical types of BeamSQL here? BeamSQL logical types are 
defined in 
https://github.com/apache/beam/blob/master/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtils.java?
 BeamSQL uses logical type to define some of its types: Date, Time, Char, etc.
   
   
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 230840)
    Time Spent: 50m  (was: 40m)

> BeamEnumerableConverter does not handle null values for all types
> -----------------------------------------------------------------
>
>                 Key: BEAM-7129
>                 URL: https://issues.apache.org/jira/browse/BEAM-7129
>             Project: Beam
>          Issue Type: Bug
>          Components: sdk-java-core
>            Reporter: Brian Hulette
>            Assignee: Brian Hulette
>            Priority: Major
>          Time Spent: 50m
>  Remaining Estimate: 0h
>
> BeamEnumerableConverter.fieldToAvatica may be called with a null beamValue, 
> but it doesn't handle this case for all types.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to