[
https://issues.apache.org/jira/browse/BEAM-7174?focusedWorklogId=240717&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-240717
]
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_r283120151
##########
File path:
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/AddFields.java
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkArgument;
+import static
org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.auto.value.AutoValue;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.schemas.FieldAccessDescriptor;
+import
org.apache.beam.sdk.schemas.FieldAccessDescriptor.FieldDescriptor.Qualifier;
+import
org.apache.beam.sdk.schemas.FieldAccessDescriptor.FieldDescriptor.Qualifier.Kind;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+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.ImmutableList;
+import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables;
+import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Lists;
+import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Maps;
+import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Multimap;
+import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Multimaps;
+
+/**
+ * A transform to add new nullable fields to a PCollection's schema. Elements
are extended to have
+ * the new schema. By default new fields are nullable, and input rows will be
extended to the new
+ * schema by inserting null values. However explicit default values for new
fields can be set using
+ * {@link Inner#field(String, Schema.FieldType, Object)}. Nested fields can be
added as well.
+ *
+ * <p>Example use:
+ *
+ * <pre>{@code PCollection<Event> events = readEvents();
+ * PCollection<Row> augmentedEvents =
+ * events.apply(AddFields.<Event>create()
+ * .field("userId", FieldType.STRING)
+ * .field("location.zipcode", FieldType.INT32)
+ * .field("userDetails.isSpecialUser", "FieldType.BOOLEAN", false));
+ * }</pre>
+ */
+public class AddFields {
+ public static <T> Inner<T> create() {
+ return new Inner<>();
+ }
+
+ /** Inner PTransform for AddFields. */
+ public static class Inner<T> extends PTransform<PCollection<T>,
PCollection<Row>> {
+ /** Internal object representing a new field added. */
+ @AutoValue
+ abstract static class NewField implements Serializable {
+ abstract String getName();
+
+ abstract FieldAccessDescriptor getDescriptor();
+
+ abstract Schema.FieldType getFieldType();
+
+ @Nullable
+ abstract Object getDefaultValue();
+
+ @AutoValue.Builder
+ abstract static class Builder {
+ abstract Builder setName(String name);
+
+ abstract Builder setDescriptor(FieldAccessDescriptor descriptor);
+
+ abstract Builder setFieldType(Schema.FieldType fieldType);
+
+ abstract Builder setDefaultValue(@Nullable Object defaultValue);
+
+ abstract NewField build();
+ }
+
+ abstract Builder toBuilder();
+
+ static NewField of(
+ FieldAccessDescriptor fieldAccessDescriptor,
+ Schema.FieldType fieldType,
+ Object defaultValue) {
+ return new AutoValue_AddFields_Inner_NewField.Builder()
+ .setName(getName(fieldAccessDescriptor))
+ .setDescriptor(fieldAccessDescriptor)
+ .setFieldType(fieldType)
+ .setDefaultValue(defaultValue)
+ .build();
+ }
+
+ // If this field represents a nested value, pop the
FieldAccessDescriptor one level down.
+ NewField descend() {
+ FieldAccessDescriptor descriptor =
+
Iterables.getOnlyElement(getDescriptor().getNestedFieldsAccessed().values());
+ return
toBuilder().setDescriptor(descriptor).setName(getName(descriptor)).build();
+ }
+
+ static String getName(FieldAccessDescriptor descriptor) {
+ if (!descriptor.getFieldsAccessed().isEmpty()) {
+ return Iterables.getOnlyElement(descriptor.fieldNamesAccessed());
+ } else {
+ return
Iterables.getOnlyElement(descriptor.nestedFieldsByName().keySet());
+ }
+ }
+
+ FieldAccessDescriptor.FieldDescriptor getFieldDescriptor() {
+ if (!getDescriptor().getFieldsAccessed().isEmpty()) {
+ return Iterables.getOnlyElement(getDescriptor().getFieldsAccessed());
+ } else {
+ return
Iterables.getOnlyElement(getDescriptor().getNestedFieldsAccessed().keySet());
+ }
+ }
+ }
+
+ /** This class encapsulates all data needed to add a a new field to the
schema. */
+ @AutoValue
+ abstract static class AddFieldsInformation implements Serializable {
+ // The new output fieldtype after adding the new field.
+ @Nullable
+ abstract Schema.FieldType getOutputFieldType();
+
+ // A list of default values corresponding to this level of the schema.
+ abstract List<Object> getDefaultValues();
+
+ // A list of nested values. This list corresponds to the output schema
fields, and is
+ // populated for fields that
+ // have new nested values. For other fields, the list contains a null
value.
+ abstract List<AddFieldsInformation> getNestedNewValues();
+
+ @AutoValue.Builder
+ abstract static class Builder {
+ abstract AddFieldsInformation.Builder
setOutputFieldType(Schema.FieldType outputFieldType);
+
+ abstract AddFieldsInformation.Builder setDefaultValues(List<Object>
defaultValues);
+
+ abstract AddFieldsInformation.Builder setNestedNewValues(
+ List<AddFieldsInformation> nestedNewValues);
+
+ abstract AddFieldsInformation build();
+ }
+
+ abstract AddFieldsInformation.Builder toBuilder();
+
+ static AddFieldsInformation of(
+ Schema.FieldType outputFieldType,
+ List<Object> defaultValues,
+ List<AddFieldsInformation> nestedNewValues) {
+ return new AutoValue_AddFields_Inner_AddFieldsInformation.Builder()
+ .setOutputFieldType(outputFieldType)
+ .setDefaultValues(defaultValues)
+ .setNestedNewValues(nestedNewValues)
+ .build();
+ }
+ }
+
+ private final List<NewField> newFields;
+
+ private Inner() {
+ this.newFields = Collections.emptyList();
+ }
+
+ private Inner(List<NewField> newFields) {
+ this.newFields = newFields;
+ }
+
+ /**
+ * Add a new field of the specified type. The new field will be nullable
and will be filled in
+ * with null values.
+ */
+ public Inner<T> field(String fieldName, Schema.FieldType fieldType) {
+ return field(fieldName, fieldType.withNullable(true), null);
+ }
+
+ /**
+ * Add a new field of the specified type. The new field will be filled in
with the specified
+ * value.
+ */
+ public Inner<T> field(String fieldName, Schema.FieldType fieldType, Object
defaultValue) {
+ if (defaultValue == null) {
+ checkArgument(fieldType.getNullable());
+ }
+
+ FieldAccessDescriptor descriptor =
FieldAccessDescriptor.withFieldNames(fieldName);
+ checkArgument(descriptor.referencesSingleField());
+ List<NewField> fields =
+ ImmutableList.<NewField>builder()
+ .addAll(newFields)
+ .add(NewField.of(descriptor, fieldType, defaultValue))
+ .build();
+ return new Inner<>(fields);
+ }
+
+ private AddFieldsInformation getAddFieldsInformation(
Review comment:
I believe these functions can be static.
----------------------------------------------------------------
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: 240717)
Time Spent: 40m (was: 0.5h)
> 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: 40m
> 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)