All Downloads are FREE. Search and download functionalities are using the official Maven repository.

org.apache.hudi.avro.AvroSchemaUtils Maven / Gradle / Ivy

There is a newer version: 1.0.0-beta2
Show newest version
/*
 * 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.hudi.avro;

import org.apache.avro.AvroRuntimeException;
import org.apache.avro.Schema;
import org.apache.avro.SchemaCompatibility;

import java.util.List;
import java.util.Objects;
import java.util.function.BiFunction;
import java.util.stream.Collectors;

import static org.apache.hudi.common.util.ValidationUtils.checkState;

/**
 * Utils for Avro Schema.
 */
public class AvroSchemaUtils {

  private AvroSchemaUtils() {}

  /**
   * See {@link #isSchemaCompatible(Schema, Schema, boolean, boolean)} doc for more details
   */
  public static boolean isSchemaCompatible(Schema prevSchema, Schema newSchema, boolean allowProjection) {
    return isSchemaCompatible(prevSchema, newSchema, true, allowProjection);
  }

  /**
   * Establishes whether {@code prevSchema} is compatible w/ {@code newSchema}, as
   * defined by Avro's {@link AvroSchemaCompatibility}
   *
   * @param prevSchema previous instance of the schema
   * @param newSchema new instance of the schema
   * @param checkNaming controls whether schemas fully-qualified names should be checked
   */
  public static boolean isSchemaCompatible(Schema prevSchema, Schema newSchema, boolean checkNaming, boolean allowProjection) {
    // NOTE: We're establishing compatibility of the {@code prevSchema} and {@code newSchema}
    //       as following: {@code newSchema} is considered compatible to {@code prevSchema},
    //       iff data written using {@code prevSchema} could be read by {@code newSchema}

    // In case schema projection is not allowed, new schema has to have all the same fields as the
    // old schema
    if (!allowProjection) {
      if (!canProject(prevSchema, newSchema)) {
        return false;
      }
    }

    AvroSchemaCompatibility.SchemaPairCompatibility result =
        AvroSchemaCompatibility.checkReaderWriterCompatibility(newSchema, prevSchema, checkNaming);
    return result.getType() == AvroSchemaCompatibility.SchemaCompatibilityType.COMPATIBLE;
  }

  /**
   * Check that each field in the prevSchema can be populated in the newSchema
   * @param prevSchema prev schema.
   * @param newSchema new schema
   * @return true if prev schema is a projection of new schema.
   */
  public static boolean canProject(Schema prevSchema, Schema newSchema) {
    return prevSchema.getFields().stream()
        .map(oldSchemaField -> SchemaCompatibility.lookupWriterField(newSchema, oldSchemaField))
        .noneMatch(Objects::isNull);
  }

  /**
   * Generates fully-qualified name for the Avro's schema based on the Table's name
   *
   * NOTE: PLEASE READ CAREFULLY BEFORE CHANGING
   *       This method should not change for compatibility reasons as older versions
   *       of Avro might be comparing fully-qualified names rather than just the record
   *       names
   */
  public static String getAvroRecordQualifiedName(String tableName) {
    String sanitizedTableName = HoodieAvroUtils.sanitizeName(tableName);
    return "hoodie." + sanitizedTableName + "." + sanitizedTableName + "_record";
  }

  /**
   * Validate whether the {@code targetSchema} is a "compatible" projection of {@code sourceSchema}.
   *
   * Only difference of this method from {@link #isStrictProjectionOf(Schema, Schema)} is
   * the fact that it allows some legitimate type promotions (like {@code int -> long},
   * {@code decimal(3, 2) -> decimal(5, 2)}, etc) that allows projection to have a "wider"
   * atomic type (whereas strict projection requires atomic type to be identical)
   */
  public static boolean isCompatibleProjectionOf(Schema sourceSchema, Schema targetSchema) {
    return isProjectionOfInternal(sourceSchema, targetSchema,
        AvroSchemaUtils::isAtomicSchemasCompatible);
  }

  private static boolean isAtomicSchemasCompatible(Schema oneAtomicType, Schema anotherAtomicType) {
    // NOTE: Checking for compatibility of atomic types, we should ignore their
    //       corresponding fully-qualified names (as irrelevant)
    return isSchemaCompatible(oneAtomicType, anotherAtomicType, false, true);
  }

  /**
   * Validate whether the {@code targetSchema} is a strict projection of {@code sourceSchema}.
   *
   * Schema B is considered a strict projection of schema A iff
   * 
    *
  1. Schemas A and B are equal, or
  2. *
  3. Schemas A and B are array schemas and element-type of B is a strict projection * of the element-type of A, or
  4. *
  5. Schemas A and B are map schemas and value-type of B is a strict projection * of the value-type of A, or
  6. *
  7. Schemas A and B are union schemas (of the same size) and every element-type of B * is a strict projection of the corresponding element-type of A, or
  8. *
  9. Schemas A and B are record schemas and every field of the record B has corresponding * counterpart (w/ the same name) in the schema A, such that the schema of the field of the schema * B is also a strict projection of the A field's schema
  10. *
*/ public static boolean isStrictProjectionOf(Schema sourceSchema, Schema targetSchema) { return isProjectionOfInternal(sourceSchema, targetSchema, Objects::equals); } private static boolean isProjectionOfInternal(Schema sourceSchema, Schema targetSchema, BiFunction atomicTypeEqualityPredicate) { if (sourceSchema.getType() == targetSchema.getType()) { if (sourceSchema.getType() == Schema.Type.RECORD) { for (Schema.Field targetField : targetSchema.getFields()) { Schema.Field sourceField = sourceSchema.getField(targetField.name()); if (sourceField == null || !isProjectionOfInternal(sourceField.schema(), targetField.schema(), atomicTypeEqualityPredicate)) { return false; } } return true; } else if (sourceSchema.getType() == Schema.Type.ARRAY) { return isProjectionOfInternal(sourceSchema.getElementType(), targetSchema.getElementType(), atomicTypeEqualityPredicate); } else if (sourceSchema.getType() == Schema.Type.MAP) { return isProjectionOfInternal(sourceSchema.getValueType(), targetSchema.getValueType(), atomicTypeEqualityPredicate); } else if (sourceSchema.getType() == Schema.Type.UNION) { List sourceNestedSchemas = sourceSchema.getTypes(); List targetNestedSchemas = targetSchema.getTypes(); if (sourceNestedSchemas.size() != targetNestedSchemas.size()) { return false; } for (int i = 0; i < sourceNestedSchemas.size(); ++i) { if (!isProjectionOfInternal(sourceNestedSchemas.get(i), targetNestedSchemas.get(i), atomicTypeEqualityPredicate)) { return false; } } return true; } } return atomicTypeEqualityPredicate.apply(sourceSchema, targetSchema); } /** * Appends provided new fields at the end of the given schema * * NOTE: No deduplication is made, this method simply appends fields at the end of the list * of the source schema as is */ public static Schema appendFieldsToSchema(Schema schema, List newFields) { List fields = schema.getFields().stream() .map(field -> new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultVal())) .collect(Collectors.toList()); fields.addAll(newFields); Schema newSchema = Schema.createRecord(schema.getName(), schema.getDoc(), schema.getNamespace(), schema.isError()); newSchema.setFields(fields); return newSchema; } /** * Passed in {@code Union} schema and will try to resolve the field with the {@code fieldSchemaFullName} * w/in the union returning its corresponding schema * * @param schema target schema to be inspected * @param fieldSchemaFullName target field-name to be looked up w/in the union * @return schema of the field w/in the union identified by the {@code fieldSchemaFullName} */ public static Schema resolveUnionSchema(Schema schema, String fieldSchemaFullName) { if (schema.getType() != Schema.Type.UNION) { return schema; } List innerTypes = schema.getTypes(); Schema nonNullType = innerTypes.stream() .filter(it -> it.getType() != Schema.Type.NULL && Objects.equals(it.getFullName(), fieldSchemaFullName)) .findFirst() .orElse(null); if (nonNullType == null) { throw new AvroRuntimeException( String.format("Unsupported Avro UNION type %s: Only UNION of a null type and a non-null type is supported", schema)); } return nonNullType; } /** * Returns true in case provided {@link Schema} is nullable (ie accepting null values), * returns false otherwise */ public static boolean isNullable(Schema schema) { if (schema.getType() != Schema.Type.UNION) { return false; } List innerTypes = schema.getTypes(); return innerTypes.size() > 1 && innerTypes.stream().anyMatch(it -> it.getType() == Schema.Type.NULL); } /** * Resolves typical Avro's nullable schema definition: {@code Union(Schema.Type.NULL, )}, * decomposing union and returning the target non-null type */ public static Schema resolveNullableSchema(Schema schema) { if (schema.getType() != Schema.Type.UNION) { return schema; } List innerTypes = schema.getTypes(); Schema nonNullType = innerTypes.stream() .filter(it -> it.getType() != Schema.Type.NULL) .findFirst() .orElse(null); if (innerTypes.size() != 2 || nonNullType == null) { throw new AvroRuntimeException( String.format("Unsupported Avro UNION type %s: Only UNION of a null type and a non-null type is supported", schema)); } return nonNullType; } /** * Creates schema following Avro's typical nullable schema definition: {@code Union(Schema.Type.NULL, )}, * wrapping around provided target non-null type */ public static Schema createNullableSchema(Schema.Type avroType) { return createNullableSchema(Schema.create(avroType)); } public static Schema createNullableSchema(Schema schema) { checkState(schema.getType() != Schema.Type.NULL); return Schema.createUnion(Schema.create(Schema.Type.NULL), schema); } /** * Returns true in case when schema contains the field w/ provided name */ public static boolean containsFieldInSchema(Schema schema, String fieldName) { try { Schema.Field field = schema.getField(fieldName); return field != null; } catch (Exception e) { return false; } } }




© 2015 - 2024 Weber Informatics LLC | Privacy Policy