You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@beam.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2019/04/03 21:20:00 UTC

[jira] [Work logged] (BEAM-6772) Select transform has non-intuitive semantics

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

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

                Author: ASF GitHub Bot
            Created on: 03/Apr/19 21:19
            Start Date: 03/Apr/19 21:19
    Worklog Time Spent: 10m 
      Work Description: kanterov commented on pull request #8006: [BEAM-6772] Change Select semantics to match what a user expects
URL: https://github.com/apache/beam/pull/8006#discussion_r271936150
 
 

 ##########
 File path: sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/SelectHelpersTest.java
 ##########
 @@ -0,0 +1,345 @@
+/*
+ * 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.beam.sdk.schemas.utils;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.beam.sdk.schemas.FieldAccessDescriptor;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableMap;
+import org.junit.Test;
+
+/** Tests for {@link SelectHelpers}. */
+public class SelectHelpersTest {
+  static final Schema FLAT_SCHEMA =
+      Schema.builder()
+          .addStringField("field1")
+          .addInt32Field("field2")
+          .addDoubleField("field3")
+          .build();
+  static final Row FLAT_ROW = Row.withSchema(FLAT_SCHEMA).addValues("first", 42, 3.14).build();
+
+  static final Schema NESTED_SCHEMA =
+      Schema.builder().addRowField("nested", FLAT_SCHEMA).addStringField("foo").build();
+  static final Row NESTED_ROW = Row.withSchema(NESTED_SCHEMA).addValues(FLAT_ROW, "").build();
+
+  static final Schema DOUBLE_NESTED_SCHEMA =
+      Schema.builder().addRowField("nested2", NESTED_SCHEMA).build();
+  static final Row DOUBLE_NESTED_ROW =
+      Row.withSchema(DOUBLE_NESTED_SCHEMA).addValue(NESTED_ROW).build();
+
+  static final Schema ARRAY_SCHEMA =
+      Schema.builder()
+          .addArrayField("primitiveArray", FieldType.INT32)
+          .addArrayField("rowArray", FieldType.row(FLAT_SCHEMA))
+          .addArrayField("arrayOfRowArray", FieldType.array(FieldType.row(FLAT_SCHEMA)))
+          .addArrayField("nestedRowArray", FieldType.row(NESTED_SCHEMA))
+          .build();
+  static final Row ARRAY_ROW =
+      Row.withSchema(ARRAY_SCHEMA)
+          .addArray(1, 2)
+          .addArray(FLAT_ROW, FLAT_ROW)
+          .addArray(ImmutableList.of(FLAT_ROW), ImmutableList.of(FLAT_ROW))
+          .addArray(NESTED_ROW, NESTED_ROW)
+          .build();
+
+  static final Schema MAP_SCHEMA =
+      Schema.builder().addMapField("map", FieldType.INT32, FieldType.row(FLAT_SCHEMA)).build();
+  static final Row MAP_ROW =
+      Row.withSchema(MAP_SCHEMA).addValue(ImmutableMap.of(1, FLAT_ROW)).build();
+
+  static final Schema MAP_ARRAY_SCHEMA =
+      Schema.builder()
+          .addMapField("map", FieldType.INT32, FieldType.array(FieldType.row(FLAT_SCHEMA)))
+          .build();
+  static final Row MAP_ARRAY_ROW =
+      Row.withSchema(MAP_ARRAY_SCHEMA)
+          .addValue(ImmutableMap.of(1, ImmutableList.of(FLAT_ROW)))
+          .build();
+
+  @Test
+  public void testSelectAll() {
+    FieldAccessDescriptor fieldAccessDescriptor =
+        FieldAccessDescriptor.withFieldNames("*").resolve(FLAT_SCHEMA);
+    Schema outputSchema = SelectHelpers.getOutputSchema(FLAT_SCHEMA, fieldAccessDescriptor);
+    assertEquals(FLAT_SCHEMA, outputSchema);
+
+    Row row = SelectHelpers.selectRow(FLAT_ROW, fieldAccessDescriptor, FLAT_SCHEMA, outputSchema);
+    assertEquals(FLAT_ROW, row);
+  }
+
+  @Test
+  public void testsSimpleSelectSingle() {
+    FieldAccessDescriptor fieldAccessDescriptor =
+        FieldAccessDescriptor.withFieldNames("field1").resolve(FLAT_SCHEMA);
+    Schema outputSchema = SelectHelpers.getOutputSchema(FLAT_SCHEMA, fieldAccessDescriptor);
+    Schema expectedSchema = Schema.builder().addStringField("field1").build();
+    assertEquals(expectedSchema, outputSchema);
+
+    Row row = SelectHelpers.selectRow(FLAT_ROW, fieldAccessDescriptor, FLAT_SCHEMA, outputSchema);
+    Row expectedRow = Row.withSchema(expectedSchema).addValue("first").build();
+    assertEquals(expectedRow, row);
+  }
+
+  @Test
+  public void testsSimpleSelectMultiple() {
+    FieldAccessDescriptor fieldAccessDescriptor =
+        FieldAccessDescriptor.withFieldNames("field1", "field3").resolve(FLAT_SCHEMA);
+    Schema outputSchema = SelectHelpers.getOutputSchema(FLAT_SCHEMA, fieldAccessDescriptor);
+    Schema expectedSchema =
+        Schema.builder().addStringField("field1").addDoubleField("field3").build();
+    assertEquals(expectedSchema, outputSchema);
+
+    Row row = SelectHelpers.selectRow(FLAT_ROW, fieldAccessDescriptor, FLAT_SCHEMA, outputSchema);
+    Row expectedRow = Row.withSchema(expectedSchema).addValues("first", 3.14).build();
+    assertEquals(expectedRow, row);
+  }
+
+  @Test
+  public void testSelectedNested() {
+    FieldAccessDescriptor fieldAccessDescriptor =
+        FieldAccessDescriptor.withFieldNames("nested").resolve(NESTED_SCHEMA);
+    Schema outputSchema = SelectHelpers.getOutputSchema(NESTED_SCHEMA, fieldAccessDescriptor);
+    Schema expectedSchema = Schema.builder().addRowField("nested", FLAT_SCHEMA).build();
+    assertEquals(expectedSchema, outputSchema);
+
+    Row row =
+        SelectHelpers.selectRow(NESTED_ROW, fieldAccessDescriptor, NESTED_SCHEMA, outputSchema);
+    Row expectedRow = Row.withSchema(expectedSchema).addValue(FLAT_ROW).build();
+    assertEquals(expectedRow, row);
+  }
+
+  @Test
+  public void testSelectedNestedSingle() {
+    FieldAccessDescriptor fieldAccessDescriptor =
+        FieldAccessDescriptor.withFieldNames("nested.field1").resolve(NESTED_SCHEMA);
+    Schema outputSchema = SelectHelpers.getOutputSchema(NESTED_SCHEMA, fieldAccessDescriptor);
+    Schema expectedSchema = Schema.builder().addStringField("field1").build();
+    assertEquals(expectedSchema, outputSchema);
+
+    Row row =
+        SelectHelpers.selectRow(NESTED_ROW, fieldAccessDescriptor, NESTED_SCHEMA, outputSchema);
+    Row expectedRow = Row.withSchema(expectedSchema).addValue("first").build();
+    assertEquals(expectedRow, row);
+  }
+
+  @Test
+  public void testSelectedNestedWildcard() {
+    FieldAccessDescriptor fieldAccessDescriptor =
+        FieldAccessDescriptor.withFieldNames("nested.*").resolve(NESTED_SCHEMA);
+    Schema outputSchema = SelectHelpers.getOutputSchema(NESTED_SCHEMA, fieldAccessDescriptor);
+    assertEquals(FLAT_SCHEMA, outputSchema);
+
+    Row row =
+        SelectHelpers.selectRow(NESTED_ROW, fieldAccessDescriptor, NESTED_SCHEMA, outputSchema);
+    assertEquals(FLAT_ROW, row);
+  }
+
+  @Test
+  public void testSelectDoubleNested() {
+    FieldAccessDescriptor fieldAccessDescriptor =
+        FieldAccessDescriptor.withFieldNames("nested2.nested.field1").resolve(DOUBLE_NESTED_SCHEMA);
+    Schema outputSchema =
+        SelectHelpers.getOutputSchema(DOUBLE_NESTED_SCHEMA, fieldAccessDescriptor);
+    Schema expectedSchema = Schema.builder().addStringField("field1").build();
+    assertEquals(expectedSchema, outputSchema);
+
+    Row row =
+        SelectHelpers.selectRow(
+            DOUBLE_NESTED_ROW, fieldAccessDescriptor, DOUBLE_NESTED_SCHEMA, outputSchema);
+    Row expectedRow = Row.withSchema(expectedSchema).addValue("first").build();
+    assertEquals(expectedRow, row);
+  }
+
+  @Test
+  public void testSelectArrayOfPrimitive() {
+    FieldAccessDescriptor fieldAccessDescriptor =
+        FieldAccessDescriptor.withFieldNames("primitiveArray").resolve(ARRAY_SCHEMA);
+    Schema outputSchema = SelectHelpers.getOutputSchema(ARRAY_SCHEMA, fieldAccessDescriptor);
+    Schema expectedSchema =
+        Schema.builder().addArrayField("primitiveArray", FieldType.INT32).build();
+    assertEquals(expectedSchema, outputSchema);
+
+    Row row = SelectHelpers.selectRow(ARRAY_ROW, fieldAccessDescriptor, ARRAY_SCHEMA, outputSchema);
+    Row expectedRow = Row.withSchema(expectedSchema).addArray(1, 2).build();
+    assertEquals(expectedRow, row);
+  }
+
+  @Test
+  public void testSelectArrayOfRow() {
+    FieldAccessDescriptor fieldAccessDescriptor =
+        FieldAccessDescriptor.withFieldNames("rowArray").resolve(ARRAY_SCHEMA);
+    Schema outputSchema = SelectHelpers.getOutputSchema(ARRAY_SCHEMA, fieldAccessDescriptor);
+    Schema expectedSchema =
+        Schema.builder().addArrayField("rowArray", FieldType.row(FLAT_SCHEMA)).build();
+    assertEquals(expectedSchema, outputSchema);
+
+    Row row = SelectHelpers.selectRow(ARRAY_ROW, fieldAccessDescriptor, ARRAY_SCHEMA, outputSchema);
+    Row expectedRow = Row.withSchema(expectedSchema).addArray(FLAT_ROW, FLAT_ROW).build();
+    assertEquals(expectedRow, row);
+  }
+
+  @Test
+  public void testSelectArrayOfRowPartial() {
+    FieldAccessDescriptor fieldAccessDescriptor =
+        FieldAccessDescriptor.withFieldNames("rowArray[].field1").resolve(ARRAY_SCHEMA);
+    Schema outputSchema = SelectHelpers.getOutputSchema(ARRAY_SCHEMA, fieldAccessDescriptor);
+
+    Schema expectedElementSchema = Schema.builder().addStringField("field1").build();
+    Schema expectedSchema =
+        Schema.builder().addArrayField("rowArray", FieldType.row(expectedElementSchema)).build();
+    assertEquals(expectedSchema, outputSchema);
+
+    Row row = SelectHelpers.selectRow(ARRAY_ROW, fieldAccessDescriptor, ARRAY_SCHEMA, outputSchema);
+
+    Row expectedElement = Row.withSchema(expectedElementSchema).addValue("first").build();
+    Row expectedRow =
+        Row.withSchema(expectedSchema).addArray(expectedElement, expectedElement).build();
+    assertEquals(expectedRow, row);
+  }
+
+  @Test
+  public void testSelectArrayOfRowArray() {
+    FieldAccessDescriptor fieldAccessDescriptor =
+        FieldAccessDescriptor.withFieldNames("arrayOfRowArray[][].field1").resolve(ARRAY_SCHEMA);
+    Schema outputSchema = SelectHelpers.getOutputSchema(ARRAY_SCHEMA, fieldAccessDescriptor);
+
+    Schema expectedElementSchema = Schema.builder().addStringField("field1").build();
+    Schema expectedSchema =
+        Schema.builder()
+            .addArrayField("arrayOfRowArray", FieldType.array(FieldType.row(expectedElementSchema)))
+            .build();
+    assertEquals(expectedSchema, outputSchema);
+
+    Row row = SelectHelpers.selectRow(ARRAY_ROW, fieldAccessDescriptor, ARRAY_SCHEMA, outputSchema);
+
+    Row expectedElement = Row.withSchema(expectedElementSchema).addValue("first").build();
+    Row expectedRow =
+        Row.withSchema(expectedSchema)
+            .addArray(ImmutableList.of(expectedElement), ImmutableList.of(expectedElement))
+            .build();
+    assertEquals(expectedRow, row);
+  }
+
+  @Test
+  public void testSelectArrayOfNestedRow() {
+    FieldAccessDescriptor fieldAccessDescriptor =
+        FieldAccessDescriptor.withFieldNames("nestedRowArray[].nested.field1")
+            .resolve(ARRAY_SCHEMA);
+    Schema outputSchema = SelectHelpers.getOutputSchema(ARRAY_SCHEMA, fieldAccessDescriptor);
+
+    Schema expectedElementSchema = Schema.builder().addStringField("field1").build();
+    Schema expectedSchema =
+        Schema.builder()
+            .addArrayField("nestedRowArray", FieldType.row(expectedElementSchema))
+            .build();
+    assertEquals(expectedSchema, outputSchema);
+
+    Row row = SelectHelpers.selectRow(ARRAY_ROW, fieldAccessDescriptor, ARRAY_SCHEMA, outputSchema);
+
+    Row expectedElement = Row.withSchema(expectedElementSchema).addValue("first").build();
+    Row expectedRow =
+        Row.withSchema(expectedSchema).addArray(expectedElement, expectedElement).build();
+    assertEquals(expectedRow, row);
+  }
+
+  @Test
+  public void testSelectMapOfRowSelectSingle() {
+    FieldAccessDescriptor fieldAccessDescriptor =
+        FieldAccessDescriptor.withFieldNames("map{}.field1").resolve(MAP_SCHEMA);
+    Schema outputSchema = SelectHelpers.getOutputSchema(MAP_SCHEMA, fieldAccessDescriptor);
+
+    Schema expectedValueSchema = Schema.builder().addStringField("field1").build();
+    Schema expectedSchema =
+        Schema.builder()
+            .addMapField("map", FieldType.INT32, FieldType.row(expectedValueSchema))
+            .build();
+    assertEquals(expectedSchema, outputSchema);
+
+    Row row = SelectHelpers.selectRow(MAP_ROW, fieldAccessDescriptor, MAP_SCHEMA, outputSchema);
+
+    Row expectedValueRow = Row.withSchema(expectedValueSchema).addValue("first").build();
+    Row expectedRow =
+        Row.withSchema(expectedSchema).addValue(ImmutableMap.of(1, expectedValueRow)).build();
+    assertEquals(expectedRow, row);
+  }
+
+  @Test
+  public void testSelectMapOfRowSelectAll() {
+    FieldAccessDescriptor fieldAccessDescriptor =
+        FieldAccessDescriptor.withFieldNames("map{}.*").resolve(MAP_SCHEMA);
+    Schema outputSchema = SelectHelpers.getOutputSchema(MAP_SCHEMA, fieldAccessDescriptor);
+    Schema expectedSchema =
+        Schema.builder().addMapField("map", FieldType.INT32, FieldType.row(FLAT_SCHEMA)).build();
+    assertEquals(expectedSchema, outputSchema);
+
+    Row row = SelectHelpers.selectRow(MAP_ROW, fieldAccessDescriptor, MAP_SCHEMA, outputSchema);
+    Row expectedRow = Row.withSchema(expectedSchema).addValue(ImmutableMap.of(1, FLAT_ROW)).build();
+    assertEquals(expectedRow, row);
+  }
+
+  @Test
+  public void testSelectMapOfArray() {
+    FieldAccessDescriptor fieldAccessDescriptor =
+        FieldAccessDescriptor.withFieldNames("map.field1").resolve(MAP_ARRAY_SCHEMA);
+    Schema outputSchema = SelectHelpers.getOutputSchema(MAP_ARRAY_SCHEMA, fieldAccessDescriptor);
+
+    Schema expectedValueSchema = Schema.builder().addStringField("field1").build();
+    Schema expectedSchema =
+        Schema.builder()
+            .addMapField(
+                "map", FieldType.INT32, FieldType.array(FieldType.row(expectedValueSchema)))
+            .build();
+    assertEquals(expectedSchema, outputSchema);
+
+    Row row =
+        SelectHelpers.selectRow(
+            MAP_ARRAY_ROW, fieldAccessDescriptor, MAP_ARRAY_SCHEMA, outputSchema);
+    Row expectedElement = Row.withSchema(expectedValueSchema).addValue("first").build();
+
+    Row expectedRow =
+        Row.withSchema(expectedSchema)
+            .addValue(ImmutableMap.of(1, ImmutableList.of(expectedElement)))
+            .build();
+    assertEquals(expectedRow, row);
+  }
+
+  @Test
+  public void testSelectFieldOfRecord() {
+    Schema f1 = Schema.builder().addInt64Field("f0").build();
+    Schema f2 = Schema.builder().addRowField("f1", f1).build();
+    Schema f3 = Schema.builder().addRowField("f2", f2).build();
+
+    Row r1 = Row.withSchema(f1).addValue(42L).build(); // {"f0": 42}
+    Row r2 = Row.withSchema(f2).addValue(r1).build(); // {"f1": {"f0": 42}}
+    Row r3 = Row.withSchema(f3).addValue(r2).build(); // {"f2": {"f1": {"f0": 42}}}
+
+    FieldAccessDescriptor fieldAccessDescriptor =
+        FieldAccessDescriptor.withFieldNames("f2.f1").resolve(f3);
+
+    Schema outputSchema = SelectHelpers.getOutputSchema(f3, fieldAccessDescriptor);
+
+    Row out = SelectHelpers.selectRow(r3, fieldAccessDescriptor, r3.getSchema(), outputSchema);
+
+    assertEquals(outputSchema, f2);
+    assertEquals(out, r2);
 
 Review comment:
   Parameters should be in a different order: expected, actual
 
----------------------------------------------------------------
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:
users@infra.apache.org


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

    Worklog Id:     (was: 222629)
    Time Spent: 10h 20m  (was: 10h 10m)

> Select transform has non-intuitive semantics
> --------------------------------------------
>
>                 Key: BEAM-6772
>                 URL: https://issues.apache.org/jira/browse/BEAM-6772
>             Project: Beam
>          Issue Type: Sub-task
>          Components: sdk-java-core
>            Reporter: Reuven Lax
>            Assignee: Reuven Lax
>            Priority: Major
>          Time Spent: 10h 20m
>  Remaining Estimate: 0h
>
> Consider the following schema:
> User:
>     name: STRING
>     location: Location
>  
> Location:
>     latitude: DOUBLE
>     longitude: DOUBLE
>  
> If you apply Select.fieldNames("location"), most users expect to get back a row matching the Location schema. Instead you get back an outer schema with a single location field in it. Select should instead unnest the output up to the point where multiple fields are selected.



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