[ https://issues.apache.org/jira/browse/BEAM-4461?focusedWorklogId=143639&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-143639 ]
ASF GitHub Bot logged work on BEAM-4461: ---------------------------------------- Author: ASF GitHub Bot Created on: 12/Sep/18 17:10 Start Date: 12/Sep/18 17:10 Worklog Time Spent: 10m Work Description: reuvenlax commented on a change in pull request #6316: [BEAM-4461] Add Unnest transform. URL: https://github.com/apache/beam/pull/6316#discussion_r217115670 ########## File path: sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Unnest.java ########## @@ -0,0 +1,182 @@ +/* + * 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 com.google.auto.value.AutoValue; +import com.google.common.collect.Lists; +import java.util.List; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Experimental.Kind; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.Schema.Field; +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.transforms.SerializableFunction; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; +/** + * A {@link PTransform} to unnest nested rows. + * + * <p>For example, consider a Row with the following nestedschema: + * + * <p>UserEvent Schema: userid: INT64 timestamp: DATETIME location: LatLong + * + * <p>LatLong Schema: latitude: DOUBLE longitude: DOUBLE + * + * <p>After unnesting, all of the rows will be converted to rows satisfying the following schema: + * + * <p>UserEvent Schema: userid: INT64 timestamp: DATETIME location.latitude: DOUBLE + * location.longitude: DOUBLE + * + * <p>By default nested names are concatenated to generated the unnested name, however {@link + * Unnest.Inner#withFieldNameFunction} can be used to specify a custom naming policy. + * + * <p>Note that currently array and map values are not unnested. + */ +@Experimental(Kind.SCHEMAS) +public class Unnest { + public static <T> Inner<T> create() { + return new AutoValue_Unnest_Inner.Builder<T>() + .setMaxLevels(Integer.MAX_VALUE) + .setFieldNameFunction(CONCAT_FIELD_NAMES) + .build(); + } + /** + * This is the default naming policy for naming fields. Every field name in the path to a given + * field is concated with . characters. + */ + public static final SerializableFunction<List<String>, String> CONCAT_FIELD_NAMES = + l -> { + return String.join(".", l); + }; + /** + * This policy keeps the raw nested field name. If two differently-nested fields have the same + * name, unnesting will fail with this policy. + */ + public static final SerializableFunction<List<String>, String> KEEP_NESTED_NAME = Review comment: oh, no - I plan on always defaulting to fully-qualified names. If the user wants simpler names (e.g. to implement something like select x.y as foo), it's up to them to provide the function tp do that. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 143639) Time Spent: 8h 40m (was: 8.5h) > Create a library of useful transforms that use schemas > ------------------------------------------------------ > > Key: BEAM-4461 > URL: https://issues.apache.org/jira/browse/BEAM-4461 > Project: Beam > Issue Type: Sub-task > Components: sdk-java-core > Reporter: Reuven Lax > Assignee: Reuven Lax > Priority: Major > Time Spent: 8h 40m > Remaining Estimate: 0h > > e.g. JoinBy(fields). Project, Filter, etc. -- This message was sent by Atlassian JIRA (v7.6.3#76005)