[ https://issues.apache.org/jira/browse/BEAM-5918?focusedWorklogId=162654&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-162654 ]
ASF GitHub Bot logged work on BEAM-5918: ---------------------------------------- Author: ASF GitHub Bot Created on: 05/Nov/18 18:23 Start Date: 05/Nov/18 18:23 Worklog Time Spent: 10m Work Description: kennknowles commented on a change in pull request #6888: [BEAM-5918] Add Cast transform for Rows URL: https://github.com/apache/beam/pull/6888#discussion_r230857029 ########## File path: sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Cast.java ########## @@ -0,0 +1,440 @@ +/* + * 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.base.Joiner; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Maps; +import java.io.Serializable; +import java.math.BigDecimal; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.schemas.FieldAccessDescriptor; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.Schema.Field; +import org.apache.beam.sdk.schemas.Schema.FieldType; +import org.apache.beam.sdk.schemas.Schema.TypeName; +import org.apache.beam.sdk.schemas.utils.SchemaZipFold; +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; + +/** Set of utilities for casting rows between schemas. */ +@Experimental(Experimental.Kind.SCHEMAS) +@AutoValue +public abstract class Cast<T> extends PTransform<PCollection<T>, PCollection<Row>> { + + public abstract Schema outputSchema(); + + public abstract Validator validator(); + + public static <T> Cast<T> of(Schema outputSchema, Validator validator) { + return new AutoValue_Cast<>(outputSchema, validator); + } + + public static <T> Cast<T> widening(Schema outputSchema) { + return new AutoValue_Cast<>(outputSchema, Widening.of()); + } + + public static <T> Cast<T> narrowing(Schema outputSchema) { + return new AutoValue_Cast<>(outputSchema, Narrowing.of()); + } + + /** Describes compatibility errors during casting. */ + @AutoValue + public abstract static class CompatibilityError implements Serializable { + + public abstract List<String> path(); + + public abstract String message(); + + public static CompatibilityError create(List<String> path, String message) { + return new AutoValue_Cast_CompatibilityError(path, message); + } + } + + /** Interface for statically validating casts. */ + public interface Validator extends Serializable { + List<CompatibilityError> apply(Schema input, Schema output); + } + + /** + * Widening changes to type that can represent any possible value of the original type. + * + * <p>Standard widening conversions: + * + * <ul> + * <li>BYTE to INT16, INT32, INT64, FLOAT, DOUBLE, DECIMAL + * <li>INT16 to INT32, INT64, FLOAT, DOUBLE, DECIMAL + * <li>INT32 to INT64, FLOAT, DOUBLE, DECIMAL + * <li>INT64 to FLOAT, DOUBLE, DECIMAL + * <li>FLOAT to DOUBLE, DECIMAL + * <li>DOUBLE to DECIMAL + * </ul> + * + * <p>Row widening: + * + * <ul> + * <li>wider schema to schema with a subset of fields + * <li>non-nullable fields to nullable fields + * </ul> + * + * <p>Widening doesn't lose information about the overall magnitude in following cases: + * + * <ul> + * <li>integral type to another integral type + * <li>BYTE or INT16 to FLOAT, DOUBLE or DECIMAL + * <li>INT32 to DOUBLE + * </ul> + * + * <p>Other conversions to may cause loss of precision. + */ + public static class Widening implements Validator { + private final Fold fold = new Fold(); + + public static Widening of() { + return new Widening(); + } + + @Override + public String toString() { + return "Cast.Widening"; + } + + @Override + public List<CompatibilityError> apply(final Schema input, final Schema output) { + return fold.apply(input, output); + } + + private static class Fold extends SchemaZipFold<List<CompatibilityError>> { + + @Override + public List<CompatibilityError> accumulate( + List<CompatibilityError> left, List<CompatibilityError> right) { + return ImmutableList.<CompatibilityError>builder().addAll(left).addAll(right).build(); + } + + @Override + public List<CompatibilityError> accept( + Context context, Optional<Field> left, Optional<Field> right) { + if (!left.isPresent() && !right.isPresent()) { + return Collections.emptyList(); + } else if (left.isPresent() && !right.isPresent()) { + return Collections.emptyList(); + } else if (!left.isPresent() && right.isPresent()) { + return Collections.singletonList( + CompatibilityError.create(context.path(), "Field is missing in output schema")); + } else { + if (left.get().getNullable() && !right.get().getNullable()) { + return Collections.singletonList( + CompatibilityError.create( + context.path(), "Can't cast nullable field to non-nullable field")); + } + } + + return Collections.emptyList(); + } + + @Override + public List<CompatibilityError> accept(Context context, FieldType input, FieldType output) { + TypeName inputType = input.getTypeName(); + TypeName outputType = output.getTypeName(); + + boolean supertype = outputType.isSupertypeOf(inputType); + + if (isIntegral(inputType) && isDecimal(outputType)) { Review comment: It can happen later, but I think the authoritative catalog of what is a subtype/supertype and what can be coerced can live as top-level concepts, and the transform just a quick helper to apply them. In particular, it took me a second to understand why there are two branches here. The first branch is "allowed automatic coercion" and the final return below is the usual "no action required" subtyping. ---------------------------------------------------------------- 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: 162654) Time Spent: 4h 20m (was: 4h 10m) > Add Cast transform for Rows > --------------------------- > > Key: BEAM-5918 > URL: https://issues.apache.org/jira/browse/BEAM-5918 > Project: Beam > Issue Type: Improvement > Components: sdk-java-core > Reporter: Gleb Kanterov > Assignee: Gleb Kanterov > Priority: Major > Time Spent: 4h 20m > Remaining Estimate: 0h > > There is a need for a generic transform that given two Row schemas will > convert rows between them. There must be a possibility to opt-out from > certain kind of conversions, for instance, converting ints to shorts can > cause overflow. Another example, a schema could have a nullable field, but > never have NULL value in practice, because it was filtered out. > What is needed: > - widening values (e.g., int -> long) > - narrowwing (e.g., int -> short) > - runtime check for overflow while narrowing > - ignoring nullability (nullable=true -> nullable=false) > - weakening nullability (nullable=false -> nullable=true) > - projection (Schema(a: Int32, b: Int32) -> Schema(a: Int32)) -- This message was sent by Atlassian JIRA (v7.6.3#76005)