[ https://issues.apache.org/jira/browse/BEAM-5918?focusedWorklogId=160705&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-160705 ]
ASF GitHub Bot logged work on BEAM-5918: ---------------------------------------- Author: ASF GitHub Bot Created on: 30/Oct/18 16:20 Start Date: 30/Oct/18 16:20 Worklog Time Spent: 10m Work Description: kennknowles commented on a change in pull request #6888: [BEAM-5918] [WIP] Add Cast transform for Rows URL: https://github.com/apache/beam/pull/6888#discussion_r229376846 ########## File path: sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Cast.java ########## @@ -0,0 +1,582 @@ +/* + * 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.sdk.schemas.Schema.TypeName.ARRAY; +import static org.apache.beam.sdk.schemas.Schema.TypeName.INT16; +import static org.apache.beam.sdk.schemas.Schema.TypeName.INT32; +import static org.apache.beam.sdk.schemas.Schema.TypeName.INT64; +import static org.apache.beam.sdk.schemas.Schema.TypeName.MAP; +import static org.apache.beam.sdk.schemas.Schema.TypeName.ROW; + +import com.google.auto.value.AutoValue; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Joiner; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; +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 java.util.stream.Stream; +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.TypeName; +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.KV; +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 Nullability nullability(); + + public abstract Type type(); + + public abstract Shape shape(); + + /** Builder for {@link Cast}. */ + @AutoValue.Builder + public abstract static class Builder<T> { + + public abstract Builder<T> outputSchema(Schema schema); + + public abstract Builder<T> nullability(Nullability nullability); + + public abstract Builder<T> type(Type type); + + public abstract Builder<T> shape(Shape shape); + + public abstract Cast<T> build(); + } + + public static <T> Builder<T> builder() { + return new AutoValue_Cast.Builder<T>(); + } + + public static <T> Cast<T> to(Schema outputSchema) { + return Cast.<T>builder() + .outputSchema(outputSchema) + .nullability(Nullability.IGNORE) + .type(Type.WIDEN) + .shape(Shape.PROJECTION) + .build(); + } + + public List<CompatibilityError> compatibility(Schema inputSchema) { + return Inference.compatibility(inputSchema, outputSchema(), nullability(), type(), shape()); + } + + public void verifyCompatibility(Schema inputSchema) { + List<CompatibilityError> errors = compatibility(inputSchema); + + if (!errors.isEmpty()) { + String reason = + errors + .stream() + .map(x -> Joiner.on('.').join(x.path()) + ": " + x.message()) + .collect(Collectors.joining("\n\t")); + + throw new IllegalArgumentException("Cast isn't compatible:\n\t" + reason); + } + } + + @Override + public PCollection<Row> expand(PCollection<T> input) { + Schema inputSchema = input.getSchema(); + + verifyCompatibility(inputSchema); + + return input + .apply( + ParDo.of( + new DoFn<T, Row>() { + // TODO: This should be the same as resolved so that Beam knows which fields + // are being accessed. Currently Beam only supports wildcard descriptors. + // Once BEAM-4457 is fixed, fix this. + @FieldAccess("filterFields") + final FieldAccessDescriptor fieldAccessDescriptor = + FieldAccessDescriptor.withAllFields(); + + @ProcessElement + public void process(@FieldAccess("filterFields") Row row, OutputReceiver<Row> r) { + r.output(castRow(row, inputSchema, outputSchema())); + } + })) + .setRowSchema(outputSchema()); + } + + @VisibleForTesting + static Row castRow(Row input, Schema inputSchema, Schema outputSchema) { + Row.Builder output = Row.withSchema(outputSchema); + for (int i = 0; i < outputSchema.getFieldCount(); i++) { + Schema.Field outputField = outputSchema.getField(i); + + int fromFieldIdx = inputSchema.indexOf(outputField.getName()); + Schema.Field inputField = inputSchema.getField(fromFieldIdx); + + Object inputValue = input.getValue(fromFieldIdx); + + if (inputValue == null) { + output.addValue(null); + } else { + Object value = castObject(inputValue, inputField.getType(), outputField.getType()); + output.addValue(value); + } + } + + return output.build(); + } + + @VisibleForTesting + @SuppressWarnings("unchecked") + static Object castObject( + Object inputValue, Schema.FieldType inputFieldType, Schema.FieldType outputFieldType) { + + if (inputFieldType.getTypeName() == ROW) { + return castRow( + (Row) inputValue, inputFieldType.getRowSchema(), outputFieldType.getRowSchema()); + } else if (inputFieldType.getTypeName() == ARRAY) { + List<Object> inputValues = (List<Object>) inputValue; + List<Object> outputValues = new ArrayList<>(inputValues.size()); + + for (Object elem : inputValues) { + outputValues.add( + castObject( + elem, + inputFieldType.getCollectionElementType(), + outputFieldType.getCollectionElementType())); + } + + return outputValues; + } else if (inputFieldType.getTypeName() == MAP) { + Map<Object, Object> inputMap = (Map<Object, Object>) inputValue; + Map<Object, Object> outputMap = Maps.newHashMapWithExpectedSize(inputMap.size()); + + for (Map.Entry<Object, Object> entry : inputMap.entrySet()) { + Object outputKey = + castObject( + entry.getKey(), inputFieldType.getMapKeyType(), outputFieldType.getMapKeyType()); + Object outputValue = + castObject( + entry.getValue(), + inputFieldType.getMapValueType(), + outputFieldType.getMapValueType()); + + outputMap.put(outputKey, outputValue); + } + + return outputMap; + } else { + if (inputFieldType.getTypeName().equals(outputFieldType.getTypeName())) { + return inputValue; + } else { + KV<TypeName, TypeName> key = + KV.of(inputFieldType.getTypeName(), outputFieldType.getTypeName()); + return Inference.ALL_MAP.get(key).apply(inputValue); + } + } + } + + /** Configures casting for nullable fields. */ + public enum Nullability { + /** Can cast only if nullability is the same. */ + SAME, + /** Can cast non-nullable fields to nullable. */ + WEAKEN, + /** Can cast nullable fields to non-nullable and forth. */ + IGNORE Review comment: The term `IGNORE` might be misleading. This would be a failed cast if the input is null. FWIW in this type system you can consider `NULLABLE t` to be a supertype of `t` so it is perfectly analogous to the `Type` enum. ---------------------------------------------------------------- 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: 160705) Time Spent: 50m (was: 40m) > 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: Kenneth Knowles > Priority: Major > Time Spent: 50m > 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)