Skip to content

Commit

Permalink
[HUDI-3204] Fixing partition-values being derived from partition-path…
Browse files Browse the repository at this point in the history
… instead of source columns (#5364)

 - Scaffolded `Spark24HoodieParquetFileFormat` extending `ParquetFileFormat` and overriding the behavior of adding partition columns to every row
 - Amended `SparkAdapter`s `createHoodieParquetFileFormat` API to be able to configure whether to append partition values or not
 - Fallback to append partition values in cases when the source columns are not persisted in data-file
 - Fixing HoodieBaseRelation incorrectly handling mandatory columns
  • Loading branch information
Alexey Kudinkin authored Apr 20, 2022
1 parent 408663c commit f7544e2
Show file tree
Hide file tree
Showing 28 changed files with 1,166 additions and 696 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,12 +18,30 @@
package org.apache.spark.sql

import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedFunction}
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, SubqueryExpression}
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, SubqueryExpression, UnsafeProjection}
import org.apache.spark.sql.catalyst.plans.logical.{Filter, LocalRelation, LogicalPlan}
import org.apache.spark.sql.types.StructType

trait HoodieCatalystExpressionUtils {

/**
* Generates instance of [[UnsafeProjection]] projecting row of one [[StructType]] into another [[StructType]]
*
* NOTE: No safety checks are executed to validate that this projection is actually feasible,
* it's up to the caller to make sure that such projection is possible.
*
* NOTE: Projection of the row from [[StructType]] A to [[StructType]] B is only possible, if
* B is a subset of A
*/
def generateUnsafeProjection(from: StructType, to: StructType): UnsafeProjection = {
val attrs = from.toAttributes
val attrsMap = attrs.map(attr => (attr.name, attr)).toMap
val targetExprs = to.fields.map(f => attrsMap(f.name))

GenerateUnsafeProjection.generate(targetExprs, attrs)
}

/**
* Parses and resolves expression against the attributes of the given table schema.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -177,7 +177,7 @@ trait SparkAdapter extends Serializable {
def createResolveHudiAlterTableCommand(sparkSession: SparkSession): Rule[LogicalPlan]

/**
* Create hoodie parquet file format.
* Create instance of [[ParquetFileFormat]]
*/
def createHoodieParquetFileFormat(): Option[ParquetFileFormat]
def createHoodieParquetFileFormat(appendPartitionValues: Boolean): Option[ParquetFileFormat]
}
112 changes: 112 additions & 0 deletions hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,112 @@
/*
* 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 java.util.List;
import java.util.Objects;
import java.util.stream.Collectors;

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

public class AvroSchemaUtils {

private AvroSchemaUtils() {}

/**
* 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<Schema.Field> newFields) {
List<Schema.Field> 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<Schema> 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;
}

/**
* Resolves typical Avro's nullable schema definition: {@code Union(Schema.Type.NULL, <NonNullType>)},
* decomposing union and returning the target non-null type
*/
public static Schema resolveNullableSchema(Schema schema) {
if (schema.getType() != Schema.Type.UNION) {
return schema;
}

List<Schema> 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, <NonNullType>)},
* wrapping around provided target non-null type
*/
public static Schema createNullableSchema(Schema.Type avroType) {
checkState(avroType != Schema.Type.NULL);
return Schema.createUnion(Schema.create(Schema.Type.NULL), Schema.create(avroType));
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -19,14 +19,14 @@
package org.apache.hudi.avro;

import org.apache.avro.AvroRuntimeException;
import org.apache.avro.SchemaCompatibility;
import org.apache.avro.Conversions;
import org.apache.avro.Conversions.DecimalConversion;
import org.apache.avro.JsonProperties;
import org.apache.avro.LogicalTypes;
import org.apache.avro.LogicalTypes.Decimal;
import org.apache.avro.Schema;
import org.apache.avro.Schema.Field;
import org.apache.avro.SchemaCompatibility;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericData.Record;
import org.apache.avro.generic.GenericDatumReader;
Expand Down Expand Up @@ -64,19 +64,19 @@
import java.sql.Timestamp;
import java.time.LocalDate;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.HashMap;
import java.util.TimeZone;
import java.util.Iterator;

import java.util.stream.Collectors;

import static org.apache.avro.Schema.Type.UNION;
import static org.apache.hudi.avro.AvroSchemaUtils.createNullableSchema;
import static org.apache.hudi.avro.AvroSchemaUtils.resolveNullableSchema;
import static org.apache.hudi.avro.AvroSchemaUtils.resolveUnionSchema;

/**
* Helper class to do common stuff across Avro.
Expand All @@ -97,8 +97,7 @@ public class HoodieAvroUtils {
private static final String MASK_FOR_INVALID_CHARS_IN_NAMES = "__";

// All metadata fields are optional strings.
public static final Schema METADATA_FIELD_SCHEMA =
Schema.createUnion(Arrays.asList(Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.STRING)));
public static final Schema METADATA_FIELD_SCHEMA = createNullableSchema(Schema.Type.STRING);

public static final Schema RECORD_KEY_SCHEMA = initRecordKeySchema();

Expand Down Expand Up @@ -327,31 +326,6 @@ public static GenericRecord addOperationToRecord(GenericRecord record, HoodieOpe
return record;
}

/**
* Add null fields to passed in schema. Caller is responsible for ensuring there is no duplicates. As different query
* engines have varying constraints regarding treating the case-sensitivity of fields, its best to let caller
* determine that.
*
* @param schema Passed in schema
* @param newFieldNames Null Field names to be added
*/
public static Schema appendNullSchemaFields(Schema schema, List<String> newFieldNames) {
List<Field> newFields = new ArrayList<>();
for (String newField : newFieldNames) {
newFields.add(new Schema.Field(newField, METADATA_FIELD_SCHEMA, "", JsonProperties.NULL_VALUE));
}
return createNewSchemaWithExtraFields(schema, newFields);
}

public static Schema createNewSchemaWithExtraFields(Schema schema, List<Field> newFields) {
List<Field> fields = schema.getFields().stream()
.map(field -> new 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;
}

/**
* Adds the Hoodie commit metadata into the provided Generic Record.
*/
Expand Down Expand Up @@ -736,46 +710,6 @@ public static Object getRecordColumnValues(HoodieRecord<? extends HoodieRecordPa
return getRecordColumnValues(record, columns, schema.get(), consistentLogicalTimestampEnabled);
}

private static Schema resolveUnionSchema(Schema schema, String fieldSchemaFullName) {
if (schema.getType() != Schema.Type.UNION) {
return schema;
}

List<Schema> 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;
}

public static Schema resolveNullableSchema(Schema schema) {
if (schema.getType() != Schema.Type.UNION) {
return schema;
}

List<Schema> 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;
}

/**
* Given a avro record with a given schema, rewrites it into the new schema while setting fields only from the new schema.
* support deep rewrite for nested record.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -213,7 +213,7 @@ private boolean readTextFormatMetaFile() {
format = Option.empty();
return true;
} catch (Throwable t) {
LOG.warn("Unable to read partition meta properties file for partition " + partitionPath, t);
LOG.debug("Unable to read partition meta properties file for partition " + partitionPath);
return false;
}
}
Expand All @@ -229,8 +229,7 @@ private boolean readBaseFormatMetaFile() {
format = Option.of(reader.getFormat());
return true;
} catch (Throwable t) {
// any error, log, check the next base format
LOG.warn("Unable to read partition metadata " + metafilePath.getName() + " for partition " + partitionPath, t);
LOG.debug("Unable to read partition metadata " + metafilePath.getName() + " for partition " + partitionPath);
}
}
return false;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -607,7 +607,7 @@ public String getUrlEncodePartitioning() {
return getString(URL_ENCODE_PARTITIONING);
}

public Boolean isDropPartitionColumns() {
public Boolean shouldDropPartitionColumns() {
return getBooleanOrDefault(DROP_PARTITION_COLUMNS);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,11 +23,9 @@
import org.apache.avro.Schema.Field;
import org.apache.avro.SchemaCompatibility;
import org.apache.avro.generic.IndexedRecord;

import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;

import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieFileFormat;
Expand All @@ -47,15 +45,13 @@
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIncompatibleSchemaException;
import org.apache.hudi.exception.InvalidTableException;
import org.apache.hudi.io.storage.HoodieHFileReader;
import org.apache.hudi.io.storage.HoodieOrcReader;
import org.apache.hudi.internal.schema.InternalSchema;
import org.apache.hudi.internal.schema.io.FileBasedInternalSchemaStorageManager;
import org.apache.hudi.internal.schema.utils.SerDeHelper;

import org.apache.hudi.io.storage.HoodieHFileReader;
import org.apache.hudi.io.storage.HoodieOrcReader;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;

import org.apache.parquet.avro.AvroSchemaConverter;
import org.apache.parquet.format.converter.ParquetMetadataConverter;
import org.apache.parquet.hadoop.ParquetFileReader;
Expand All @@ -67,6 +63,9 @@
import java.util.Arrays;
import java.util.List;

import static org.apache.hudi.avro.AvroSchemaUtils.appendFieldsToSchema;
import static org.apache.hudi.avro.AvroSchemaUtils.createNullableSchema;

/**
* Helper class to read schema from data files and log files and to convert it between different formats.
*
Expand Down Expand Up @@ -189,7 +188,7 @@ public Schema getTableAvroSchema(boolean includeMetadataFields) throws Exception
}

Option<String[]> partitionFieldsOpt = metaClient.getTableConfig().getPartitionFields();
if (metaClient.getTableConfig().isDropPartitionColumns()) {
if (metaClient.getTableConfig().shouldDropPartitionColumns()) {
schema = recreateSchemaWhenDropPartitionColumns(partitionFieldsOpt, schema);
}
return schema;
Expand Down Expand Up @@ -222,9 +221,9 @@ public static Schema recreateSchemaWhenDropPartitionColumns(Option<String[]> par
List<Field> newFields = new ArrayList<>();
for (String partitionField: partitionFields) {
newFields.add(new Schema.Field(
partitionField, Schema.create(Schema.Type.STRING), "", JsonProperties.NULL_VALUE));
partitionField, createNullableSchema(Schema.Type.STRING), "", JsonProperties.NULL_VALUE));
}
schema = HoodieAvroUtils.createNewSchemaWithExtraFields(schema, newFields);
schema = appendFieldsToSchema(schema, newFields);
}
}
return schema;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -89,10 +89,10 @@
import java.util.stream.Collectors;
import java.util.stream.Stream;

import static org.apache.hudi.avro.AvroSchemaUtils.resolveNullableSchema;
import static org.apache.hudi.avro.HoodieAvroUtils.addMetadataFields;
import static org.apache.hudi.avro.HoodieAvroUtils.convertValueForSpecificDataTypes;
import static org.apache.hudi.avro.HoodieAvroUtils.getNestedFieldSchemaFromWriteSchema;
import static org.apache.hudi.avro.HoodieAvroUtils.resolveNullableSchema;
import static org.apache.hudi.common.util.StringUtils.isNullOrEmpty;
import static org.apache.hudi.common.util.ValidationUtils.checkState;
import static org.apache.hudi.metadata.HoodieMetadataPayload.unwrapStatisticValueWrapper;
Expand Down
Loading

0 comments on commit f7544e2

Please sign in to comment.