[
https://issues.apache.org/jira/browse/BEAM-7174?focusedWorklogId=240721&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-240721
]
ASF GitHub Bot logged work on BEAM-7174:
----------------------------------------
Author: ASF GitHub Bot
Created on: 12/May/19 06:37
Start Date: 12/May/19 06:37
Worklog Time Spent: 10m
Work Description: robinyqiu commented on pull request #8425: [BEAM-7174]
Add schema modification transforms
URL: https://github.com/apache/beam/pull/8425#discussion_r282605059
##########
File path:
sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/DropFieldsTest.java
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.transforms;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.List;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.testing.NeedsRunner;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Lists;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/** Tests for {@link DropFields}. */
+public class DropFieldsTest {
+ @Rule public final transient TestPipeline pipeline = TestPipeline.create();
+
+ private static final Schema SIMPLE_SCHEMA =
+
Schema.builder().addInt32Field("field1").addStringField("field2").build();
+
+ private static Row simpleRow(int field1, String field2) {
+ return Row.withSchema(SIMPLE_SCHEMA).addValues(field1, field2).build();
+ }
+
+ private static final Schema NESTED_SCHEMA =
+ Schema.builder().addRowField("nested",
SIMPLE_SCHEMA).addStringField("string").build();
+
+ private static Row nestedRow(Row nested) {
+ return Row.withSchema(NESTED_SCHEMA).addValues(nested, "foo").build();
+ }
+
+ private static final Schema NESTED_ARRAY_SCHEMA =
+ Schema.builder().addArrayField("array",
FieldType.row(SIMPLE_SCHEMA)).build();
+
+ private static Row nestedArray(Row... elements) {
+ return Row.withSchema(NESTED_ARRAY_SCHEMA).addArray((Object[])
elements).build();
+ }
+
+ @Test
+ @Category(NeedsRunner.class)
+ public void testDropTopLevelField() {
+ Schema expectedSchema = Schema.builder().addStringField("field2").build();
+
+ PCollection<Row> result =
+ pipeline
+ .apply(
+ Create.of(simpleRow(1, "one"), simpleRow(2, "two"),
simpleRow(3, "three"))
+ .withRowSchema(SIMPLE_SCHEMA))
+ .apply(DropFields.fields("field1"));
+ assertEquals(expectedSchema, result.getSchema());
+
+ List<Row> expectedRows =
+ Lists.newArrayList(
+ Row.withSchema(expectedSchema).addValue("one").build(),
+ Row.withSchema(expectedSchema).addValue("two").build(),
+ Row.withSchema(expectedSchema).addValue("three").build());
+ PAssert.that(result).containsInAnyOrder(expectedRows);
+ pipeline.run();
+ }
+
+ @Test
+ @Category(NeedsRunner.class)
+ public void testDropNestedField() {
+ Schema expectedSchema =
+
Schema.builder().addStringField("string").addStringField("field2").build();
Review comment:
The original schema here is:
`nested: ROW[field1: int, field2: string], string: string`
After dropping `nested.field1`, the output schema becomes:
`string: string, field2: string`.
I have two questions here:
1) The order of the fields has changed (`string` comes before `field2` now).
Is this the intended behavior? I see this is an implementation detail in the
`SelectHelpers.union` function (It merges direct fields accessed first and then
nested fields accessed).
2) `nested.field2` is "unwrapped" to `field2`. I agree to make
`nested.field2` a top-level field in the output schema, but I think we need to
do some work to properly name the field (e.g. name it `nested.field2` instead
of only `field2`). Otherwise this may result in unexpected behaviors: e.g. if
in the example schema the second top-level field is also named `field2`, then
there will be a naming conflict and the output schema construction will fail.)
Either way, I think we need a clear documentation (and better unit tests) on
the intended behavior in the javadoc on `Select`, in addition to
`SelectHelpers`.
----------------------------------------------------------------
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: 240721)
Time Spent: 1h (was: 50m)
> Add transforms for modifying schemas
> ------------------------------------
>
> Key: BEAM-7174
> URL: https://issues.apache.org/jira/browse/BEAM-7174
> Project: Beam
> Issue Type: Sub-task
> Components: sdk-java-core
> Reporter: Reuven Lax
> Assignee: Reuven Lax
> Priority: Major
> Time Spent: 1h
> Remaining Estimate: 0h
>
> We need transforms to add fields, remove fields, and rename fields.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)