diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java index d1d98b3a584e..68d238abdcd8 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java @@ -42,8 +42,10 @@ import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.expressions.Literal; import org.apache.iceberg.expressions.UnboundTerm; +import org.apache.iceberg.mr.hive.variant.VariantPathUtil; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.transforms.Transforms; +import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.DateTimeUtil; import org.apache.iceberg.util.NaNUtil; @@ -112,8 +114,8 @@ private static Expression translate(ExpressionTree tree, List lea private static Expression translateLeaf(PredicateLeaf leaf) { TransformSpec transformSpec = TransformSpec.fromStringWithColumnName(leaf.getColumnName()); String columnName = transformSpec.getColumnName(); - UnboundTerm column = - ObjectUtils.defaultIfNull(toTerm(columnName, transformSpec), Expressions.ref(columnName)); + + UnboundTerm column = resolveTerm(columnName, leaf, transformSpec); switch (leaf.getOperator()) { case EQUALS: @@ -144,6 +146,15 @@ private static Expression translateLeaf(PredicateLeaf leaf) { } } + private static UnboundTerm resolveTerm(String columnName, PredicateLeaf leaf, TransformSpec transformSpec) { + UnboundTerm column = tryVariantExtractTerm(columnName, leaf); + if (column != null) { + return column; + } + return ObjectUtils.defaultIfNull( + toTerm(columnName, transformSpec), Expressions.ref(columnName)); + } + public static UnboundTerm toTerm(String columnName, TransformSpec transformSpec) { if (transformSpec == null) { return null; @@ -168,6 +179,66 @@ public static UnboundTerm toTerm(String columnName, TransformSpec transf } } + /** + * Converts a shredded variant pseudo-column (e.g. {@code data.typed_value.age}) into an Iceberg variant extract term + * (e.g. {@code extract(data, "$.age", "long")}). + * + *

This enables Iceberg to prune manifests/files using variant metrics produced when variant shredding is enabled. + */ + private static UnboundTerm tryVariantExtractTerm(String columnName, PredicateLeaf leaf) { + int typedIdx = columnName.indexOf(VariantPathUtil.TYPED_VALUE_SEGMENT); + if (typedIdx < 0) { + return null; + } + + String variantColumn = columnName.substring(0, typedIdx); + String extractedPath = + columnName.substring(typedIdx + VariantPathUtil.TYPED_VALUE_SEGMENT.length()); + if (variantColumn.isEmpty() || extractedPath.isEmpty()) { + return null; + } + + Type.PrimitiveType icebergType = extractPrimitiveType(leaf); + if (icebergType == null) { + return null; + } + + // Build an RFC9535 shorthand JSONPath-like path: "$.field" or "$.a.b" + String jsonPath = "$." + extractedPath; + try { + return Expressions.extract(variantColumn, jsonPath, icebergType.toString()); + } catch (RuntimeException e) { + // Invalid path/type; fall back to normal reference handling. + return null; + } + } + + private static Type.PrimitiveType extractPrimitiveType(PredicateLeaf leaf) { + // Returned types must serialize (via toString) into Iceberg primitive type strings accepted by + // Types.fromPrimitiveString. + switch (leaf.getType()) { + case LONG: + return Types.LongType.get(); + case FLOAT: + // Hive SARG uses FLOAT for both float/double; using double is the safest default. + return Types.DoubleType.get(); + case STRING: + return Types.StringType.get(); + case BOOLEAN: + return Types.BooleanType.get(); + case DATE: + return Types.DateType.get(); + case TIMESTAMP: + // Iceberg timestamps are represented as micros in a long, but the Iceberg type is timestamp. + return Types.TimestampType.withoutZone(); + case DECIMAL: + // Precision/scale are not available in the SARG leaf type. + return null; + default: + return null; + } + } + // PredicateLeafImpl has a work-around for Kryo serialization with java.util.Date objects where it converts values to // Timestamp using Date#getTime. This conversion discards microseconds, so this is a necessary to avoid it. private static final DynFields.UnboundField LITERAL_FIELD = DynFields.builder() diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergInputFormat.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergInputFormat.java index 006cdc40668d..cc0efbf8b7a9 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergInputFormat.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergInputFormat.java @@ -54,6 +54,7 @@ import org.apache.iceberg.expressions.ResidualEvaluator; import org.apache.iceberg.hive.HiveVersion; import org.apache.iceberg.mr.InputFormatConfig; +import org.apache.iceberg.mr.hive.variant.VariantFilterRewriter; import org.apache.iceberg.mr.mapred.AbstractMapredIcebergRecordReader; import org.apache.iceberg.mr.mapred.Container; import org.apache.iceberg.mr.mapred.MapredIcebergInputFormat; @@ -91,65 +92,129 @@ public class HiveIcebergInputFormat extends MapredIcebergInputFormat } /** - * Converts the Hive filter found in the job conf to an Iceberg filter expression. - * @param conf - job conf - * @return - Iceberg data filter expression + * Encapsulates planning-time and reader-time Iceberg filter expressions derived from Hive predicates. */ - static Expression icebergDataFilterFromHiveConf(Configuration conf) { - Expression icebergFilter = SerializationUtil.deserializeFromBase64(conf.get(InputFormatConfig.FILTER_EXPRESSION)); - if (icebergFilter != null) { - // in case we already have it prepared.. - return icebergFilter; + private static final class FilterExpressions { + + private static Expression planningFilter(Configuration conf) { + // Planning-safe filter (extract removed) may already be serialized for reuse. + Expression planningFilter = SerializationUtil + .deserializeFromBase64(conf.get(InputFormatConfig.FILTER_EXPRESSION)); + if (planningFilter != null) { + // in case we already have it prepared.. + return planningFilter; + } + // Reader filter should retain extract(...) for row-group pruning. Rebuild from Hive predicate to avoid losing + // variant rewrites when planningFilter was stripped. + Expression readerFilter = icebergDataFilterFromHiveConf(conf); + if (readerFilter != null) { + return VariantFilterRewriter.stripVariantExtractPredicates(readerFilter); + } + return null; } - String hiveFilter = conf.get(TableScanDesc.FILTER_EXPR_CONF_STR); - if (hiveFilter != null) { - ExprNodeGenericFuncDesc exprNodeDesc = SerializationUtilities - .deserializeObject(hiveFilter, ExprNodeGenericFuncDesc.class); - return getFilterExpr(conf, exprNodeDesc); + + private static Expression icebergDataFilterFromHiveConf(Configuration conf) { + // Build an Iceberg filter from Hive's serialized predicate so we can preserve extract(...) terms for + // reader-level pruning (e.g. Parquet shredded VARIANT row-group pruning). + // + // This intentionally does NOT consult FILTER_EXPRESSION, because FILTER_EXPRESSION must remain safe for + // Iceberg planning-time utilities (some of which cannot stringify extract(...) terms). + String hiveFilter = conf.get(TableScanDesc.FILTER_EXPR_CONF_STR); + if (hiveFilter != null) { + ExprNodeGenericFuncDesc exprNodeDesc = + SerializationUtilities.deserializeObject(hiveFilter, ExprNodeGenericFuncDesc.class); + return getFilterExpr(conf, exprNodeDesc); + } + return null; + } + + private static Expression planningResidual(FileScanTask task, Configuration conf) { + return residual(task, conf, planningFilter(conf)); + } + + private static Expression readerResidual(FileScanTask task, Configuration conf) { + return residual(task, conf, icebergDataFilterFromHiveConf(conf)); + } + + private static Expression residual(FileScanTask task, Configuration conf, Expression filter) { + if (filter == null) { + return Expressions.alwaysTrue(); + } + boolean caseSensitive = conf.getBoolean( + InputFormatConfig.CASE_SENSITIVE, InputFormatConfig.CASE_SENSITIVE_DEFAULT); + + return ResidualEvaluator.of(task.spec(), filter, caseSensitive) + .residualFor(task.file().partition()); } - return null; } /** - * getFilterExpr extracts search argument from ExprNodeGenericFuncDesc and returns Iceberg Filter Expression + * Builds an Iceberg filter expression from a Hive predicate expression node. * @param conf - job conf * @param exprNodeDesc - Describes a GenericFunc node * @return Iceberg Filter Expression */ static Expression getFilterExpr(Configuration conf, ExprNodeGenericFuncDesc exprNodeDesc) { - if (exprNodeDesc != null) { - SearchArgument sarg = ConvertAstToSearchArg.create(conf, exprNodeDesc); - try { - return HiveIcebergFilterFactory.generateFilterExpression(sarg); - } catch (UnsupportedOperationException e) { - LOG.warn("Unable to create Iceberg filter, proceeding without it (will be applied by Hive later): ", e); + if (exprNodeDesc == null) { + return null; + } + + ExprNodeGenericFuncDesc exprForSarg = exprNodeDesc; + if (Boolean.parseBoolean(conf.get(InputFormatConfig.VARIANT_SHREDDING_ENABLED))) { + ExprNodeGenericFuncDesc rewritten = VariantFilterRewriter.rewriteForShredding(exprNodeDesc); + if (rewritten != null) { + exprForSarg = rewritten; } } - return null; + + SearchArgument sarg = ConvertAstToSearchArg.create(conf, exprForSarg); + if (sarg == null) { + return null; + } + + try { + return HiveIcebergFilterFactory.generateFilterExpression(sarg); + } catch (UnsupportedOperationException e) { + LOG.warn( + "Unable to create Iceberg filter, proceeding without it (will be applied by Hive later): ", + e); + return null; + } } /** - * Converts Hive filter found in the passed job conf to an Iceberg filter expression. Then evaluates this - * against the task's partition value producing a residual filter expression. + * Returns a residual expression that is safe to apply as a record-level filter. + * + *

This residual is derived from the task-level Iceberg planning filter (already extract-free) after + * evaluating it against the task's partition value. * @param task - file scan task to evaluate the expression against * @param conf - job conf * @return - Iceberg residual filter expression */ public static Expression residualForTask(FileScanTask task, Configuration conf) { - Expression dataFilter = icebergDataFilterFromHiveConf(conf); - if (dataFilter == null) { - return Expressions.alwaysTrue(); - } - return ResidualEvaluator.of( - task.spec(), dataFilter, - conf.getBoolean(InputFormatConfig.CASE_SENSITIVE, InputFormatConfig.CASE_SENSITIVE_DEFAULT) - ).residualFor(task.file().partition()); + return FilterExpressions.planningResidual(task, conf); + } + + /** + * Returns a residual expression intended only for reader-level pruning (best-effort). + * + *

This residual is derived from the task-level Iceberg filter after evaluating it against the task's + * partition value. It may include {@code extract(...)} predicates and is suitable for formats/readers that + * can leverage such terms for pruning (e.g. Parquet row-group pruning using shredded VARIANT columns). + * + *

Do not use this for record-level residual filtering, as {@code extract} cannot be + * evaluated at record level in Iceberg readers. + */ + public static Expression residualForReaderPruning(FileScanTask task, Configuration conf) { + return FilterExpressions.readerResidual(task, conf); } @Override public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { - Expression filter = icebergDataFilterFromHiveConf(job); + Expression filter = FilterExpressions.planningFilter(job); if (filter != null) { + // Iceberg planning-time utilities may attempt to stringify the filter. Ensure the planning filter never + // contains extract(...) or shredded typed_value references. job.set(InputFormatConfig.FILTER_EXPRESSION, SerializationUtil.serializeToBase64(filter)); } diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java index 2978341b6da5..b53b9af266fe 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java @@ -187,7 +187,6 @@ import org.apache.iceberg.mr.hive.actions.HiveIcebergDeleteOrphanFiles; import org.apache.iceberg.mr.hive.plan.IcebergBucketFunction; import org.apache.iceberg.mr.hive.udf.GenericUDFIcebergZorder; -import org.apache.iceberg.parquet.VariantUtil; import org.apache.iceberg.puffin.Blob; import org.apache.iceberg.puffin.BlobMetadata; import org.apache.iceberg.puffin.Puffin; @@ -1752,8 +1751,7 @@ private void fallbackToNonVectorizedModeBasedOnProperties(Properties tableProps) if (FileFormat.AVRO == IcebergTableUtil.defaultFileFormat(tableProps::getProperty) || isValidMetadataTable(tableProps.getProperty(IcebergAcidUtil.META_TABLE_PROPERTY)) || hasOrcTimeInSchema(tableProps, tableSchema) || - !hasParquetNestedTypeWithinListOrMap(tableProps, tableSchema) || - VariantUtil.shouldUseVariantShredding(tableProps::getProperty, tableSchema)) { + !hasParquetNestedTypeWithinListOrMap(tableProps, tableSchema)) { // disable vectorization SessionStateUtil.getQueryState(conf).ifPresent(queryState -> queryState.getConf().setBoolVar(ConfVars.HIVE_VECTORIZATION_ENABLED, false)); diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/variant/VariantFilterRewriter.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/variant/VariantFilterRewriter.java new file mode 100644 index 000000000000..1b9419fd4c84 --- /dev/null +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/variant/VariantFilterRewriter.java @@ -0,0 +1,206 @@ +/* + * 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.iceberg.mr.hive.variant; + +import java.util.List; +import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFVariantGet; +import org.apache.iceberg.expressions.And; +import org.apache.iceberg.expressions.BoundExtract; +import org.apache.iceberg.expressions.BoundPredicate; +import org.apache.iceberg.expressions.BoundReference; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.NamedReference; +import org.apache.iceberg.expressions.Not; +import org.apache.iceberg.expressions.Or; +import org.apache.iceberg.expressions.UnboundExtract; +import org.apache.iceberg.expressions.UnboundPredicate; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public final class VariantFilterRewriter { + private static final Logger LOG = LoggerFactory.getLogger(VariantFilterRewriter.class); + + private VariantFilterRewriter() { + } + + /** + * Strips {@code extract(...)} predicates and shredded {@code typed_value} column references from an + * Iceberg expression by replacing them with {@code alwaysTrue()}. + * + *

Used to make expressions safe for Iceberg's Evaluator, which cannot handle variant predicates. + * Hive applies the full predicate after reading, so correctness is preserved. + */ + public static Expression stripVariantExtractPredicates(Expression expr) { + return strip(expr); + } + + private static Expression strip(Expression expr) { + if (expr == null) { + return null; + } + + if (expr == Expressions.alwaysTrue() || expr == Expressions.alwaysFalse()) { + return expr; + } + + Expression predicateResult = stripPredicate(expr); + if (predicateResult != null) { + return predicateResult; + } + + Expression logicalResult = stripLogical(expr); + if (logicalResult != null) { + return logicalResult; + } + + // Unknown expression type: do not attempt to rewrite. + return expr; + } + + private static Expression stripPredicate(Expression expr) { + if (expr instanceof UnboundPredicate unbound) { + if (unbound.term() instanceof UnboundExtract) { + return Expressions.alwaysTrue(); + } + if (unbound.term() instanceof NamedReference ref && + ref.name() != null && + ref.name().contains(VariantPathUtil.TYPED_VALUE_SEGMENT)) { + return Expressions.alwaysTrue(); + } + return expr; + } + + if (expr instanceof BoundPredicate bound) { + if (bound.term() instanceof BoundExtract) { + return Expressions.alwaysTrue(); + } + if (bound.term() instanceof BoundReference ref && + ref.name() != null && + ref.name().contains(VariantPathUtil.TYPED_VALUE_SEGMENT)) { + return Expressions.alwaysTrue(); + } + return expr; + } + + return null; + } + + private static Expression stripLogical(Expression expr) { + if (expr instanceof Not not) { + Expression child = strip(not.child()); + // alwaysTrue() means "unknown" after stripping, NOT(unknown) is still unknown + if (child == Expressions.alwaysTrue()) { + return Expressions.alwaysTrue(); + } + return Expressions.not(child); + } + + if (expr instanceof And and) { + return Expressions.and(strip(and.left()), strip(and.right())); + } + + if (expr instanceof Or or) { + return Expressions.or(strip(or.left()), strip(or.right())); + } + + return null; + } + + public static ExprNodeGenericFuncDesc rewriteForShredding(ExprNodeGenericFuncDesc predicate) { + if (predicate == null) { + return null; + } + + ExprNodeGenericFuncDesc cloned = (ExprNodeGenericFuncDesc) predicate.clone(); + ExprNodeDesc rewrittenRoot = rewriteNode(cloned); + if (rewrittenRoot instanceof ExprNodeGenericFuncDesc) { + return (ExprNodeGenericFuncDesc) rewrittenRoot; + } + + // If rewrites ended up replacing the root, fall back to the original predicate clone to avoid + // changing the expected root type. + return cloned; + } + + private static ExprNodeDesc rewriteNode(ExprNodeDesc node) { + if (node == null) { + return null; + } + + if (node instanceof ExprNodeGenericFuncDesc funcDesc) { + List children = funcDesc.getChildren(); + if (children != null) { + children.replaceAll(VariantFilterRewriter::rewriteNode); + } + + if (isVariantGet(funcDesc)) { + ExprNodeDesc replacement = rewriteVariantFunction(funcDesc); + if (replacement != null) { + return replacement; + } + } + } + + return node; + } + + private static boolean isVariantGet(ExprNodeGenericFuncDesc funcDesc) { + return funcDesc.getGenericUDF() instanceof GenericUDFVariantGet; + } + + private static ExprNodeDesc rewriteVariantFunction(ExprNodeGenericFuncDesc funcDesc) { + List args = funcDesc.getChildren(); + if (args == null || args.size() < 2) { + return null; + } + + ExprNodeDesc variantColumn = args.get(0); + ExprNodeDesc jsonPathDesc = args.get(1); + + if (!(variantColumn instanceof ExprNodeColumnDesc variantColumnDesc) || + !(jsonPathDesc instanceof ExprNodeConstantDesc)) { + return null; + } + + Object literal = ((ExprNodeConstantDesc) jsonPathDesc).getValue(); + if (!(literal instanceof String)) { + return null; + } + + List fieldPath = VariantPathUtil.parseSimpleObjectPath((String) literal); + if (fieldPath == null || fieldPath.isEmpty()) { + return null; + } + + String shreddedColumn = VariantPathUtil.shreddedColumnPath(variantColumnDesc.getColumn(), fieldPath); + LOG.debug("Rewriting variant predicate to use shredded column {}", shreddedColumn); + + return new ExprNodeColumnDesc( + funcDesc.getTypeInfo(), + shreddedColumn, + variantColumnDesc.getTabAlias(), + variantColumnDesc.getIsPartitionColOrVirtualCol()); + } +} diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/variant/VariantPathUtil.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/variant/VariantPathUtil.java new file mode 100644 index 000000000000..12256df2f99c --- /dev/null +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/variant/VariantPathUtil.java @@ -0,0 +1,116 @@ +/* + * 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.iceberg.mr.hive.variant; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import org.apache.iceberg.expressions.UnboundExtract; +import org.apache.iceberg.expressions.UnboundPredicate; + +public final class VariantPathUtil { + + // Variant field names (matching ParquetVariantVisitor package-protected constants) + public static final String METADATA = "metadata"; + public static final String VALUE = "value"; + public static final String TYPED_VALUE = "typed_value"; + + public static final String TYPED_VALUE_SEGMENT = ".typed_value."; + + private VariantPathUtil() { + } + + /** + * Parse a simple object-only JSON path (RFC9535 shorthand, e.g. {@code $.a} or {@code $.a.b}) into path segments. + * Returns {@code null} when the path is unsupported (arrays, wildcards, recursive descent, or empty). + */ + public static List parseSimpleObjectPath(String jsonPath) { + if (jsonPath == null) { + return null; + } + + String trimmed = jsonPath.trim(); + if (!trimmed.startsWith("$.") || trimmed.length() <= 2) { + return null; + } + + String remaining = trimmed.substring(2); + if (remaining.contains("[") || remaining.contains("]") || remaining.contains("*") || remaining.contains("..")) { + return null; + } + + List segments = splitPath(remaining); + return segments.isEmpty() ? null : segments; + } + + /** + * Builds a shredded pseudo-column path from a base variant column and path segments (e.g. base {@code payload}, + * segments {@code ["a", "b"]} -> {@code payload.typed_value.a.b}). + */ + public static String shreddedColumnPath(String variantColumn, List segments) { + Objects.requireNonNull(variantColumn, "variantColumn"); + if (segments == null || segments.isEmpty()) { + throw new IllegalArgumentException("segments must not be empty for shredded column path"); + } + return variantColumn + TYPED_VALUE_SEGMENT + String.join(".", segments); + } + + /** + * Splits a dot-delimited path into segments, ignoring empty segments. + * @param path the path string (e.g. {@code a.b} without a leading {@code $.}) + */ + public static List splitPath(String path) { + if (path == null || path.isEmpty()) { + return Collections.emptyList(); + } + return Arrays.stream(path.split("\\.")).filter(s -> !s.isEmpty()) + .toList(); + } + + /** + * Detects a variant shredded column reference from an unbound predicate: either a typed_value column name, or an + * extract(...) term. + */ + public static String extractVariantShreddedColumn(UnboundPredicate predicate) { + // Common case: predicate already uses shredded column references (e.g. "payload.typed_value.tier"). + String column = predicate.ref().name(); + if (column != null && column.contains(TYPED_VALUE_SEGMENT)) { + return column; + } + + // Variant extract case: rewrite extract(payload, "$.tier", "string") into "payload.typed_value.tier". + if (predicate.term() instanceof UnboundExtract extract) { + String base = extract.ref().name(); + if (base == null || base.isEmpty()) { + return null; + } + + List segments = parseSimpleObjectPath(extract.path()); + if (segments == null || segments.isEmpty()) { + return null; + } + + return shreddedColumnPath(base, segments); + } + + return column; + } +} diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/variant/VariantProjectionUtil.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/variant/VariantProjectionUtil.java new file mode 100644 index 000000000000..3372c64a61cb --- /dev/null +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/variant/VariantProjectionUtil.java @@ -0,0 +1,497 @@ +/* + * 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.iceberg.mr.hive.variant; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.io.IOConstants; +import org.apache.hadoop.hive.ql.io.parquet.read.DataWritableReadSupport; +import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; +import org.apache.iceberg.Schema; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.Types; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.OriginalType; +import org.apache.parquet.schema.Type; + +/** + * Utility class for handling Variant Projection Pushdown (Column Pruning). + */ +public final class VariantProjectionUtil { + + private VariantProjectionUtil() { + } + + public record VariantColumnDescriptor( + int rootColumnIndex, + int[] fieldPath, + String[] physicalPath, + Type prunedSchema) { + } + + public record VariantProjection( + List variantColumns, + List requestedColumns) { + + public static VariantProjection create( + MessageType fileSchema, Configuration conf, Schema icebergSchema) { + String columns = conf.get(IOConstants.COLUMNS); + if (columns == null || columns.isEmpty()) { + return null; + } + + List columnNames = DataWritableReadSupport.getColumnNames(columns); + if (columnNames == null || columnNames.isEmpty()) { + return null; + } + + boolean readAll = ColumnProjectionUtils.isReadAllColumns(conf); + List readColumnIds = ColumnProjectionUtils.getReadColumnIDs(conf); + Set nestedPaths = ColumnProjectionUtils.getNestedColumnPaths(conf); + + List variantColumns = + discoverVariantColumns(fileSchema, icebergSchema, columnNames, readAll, readColumnIds, nestedPaths); + if (variantColumns.isEmpty()) { + return null; + } + + List requestedColumns = computeRequestedColumns(fileSchema, variantColumns); + if (requestedColumns.isEmpty()) { + return null; + } + + return new VariantProjection(variantColumns, requestedColumns); + } + } + + private static List discoverVariantColumns( + MessageType fileSchema, + Schema icebergSchema, + List columnNames, + boolean readAll, + List readColumnIds, + Set nestedPaths) { + + boolean[] projected = projectedTopLevelColumns(readAll, readColumnIds, columnNames.size()); + + List result = Lists.newArrayList(); + for (int colIndex = 0; colIndex < columnNames.size(); colIndex++) { + if (!projected[colIndex]) { + continue; + } + + String columnName = columnNames.get(colIndex); + // Resolve the logical Iceberg field (handling potential name mismatches from Hive) + Types.NestedField field = findIcebergField(icebergSchema, columnName, colIndex); + if (field == null || field.type() == null) { + continue; + } + + // Resolve the physical Parquet type (handling schema evolution/IDs) + Type parquetType = findParquetType(fileSchema, field); + if (parquetType == null) { + continue; + } + + collectVariantColumns( + colIndex, + field.type(), + parquetType, + new int[0], + new String[] { columnName }, + new String[] { parquetType.getName() }, + result, + nestedPaths); + } + + return result; + } + + /** + * Resolves the Iceberg field for the given Hive column. + * Prioritizes lookup by name, falling back to positional index if not found (e.g. rename). + */ + private static Types.NestedField findIcebergField(Schema schema, String name, int index) { + Types.NestedField field = schema.findField(name); + if (field != null) { + return field; + } + + // Fallback: If Hive's column configuration uses an old name (e.g. "payload") but the Iceberg schema + // uses the new name (e.g. "data"), try to match by position if the index is valid. + List columns = schema.columns(); + if (index >= 0 && index < columns.size()) { + return columns.get(index); + } + + return null; + } + + /** + * Resolves the Parquet Type for the given Iceberg field. + * Prioritizes lookup by Field ID (for schema evolution), falling back to name. + */ + private static Type findParquetType(GroupType parent, Types.NestedField child) { + int id = child.fieldId(); + for (Type candidate : parent.getFields()) { + if (candidate.getId() != null && candidate.getId().intValue() == id) { + return candidate; + } + } + + String name = child.name(); + if (name != null && !name.isEmpty() && parent.containsField(name)) { + return parent.getType(name); + } + + return null; + } + + private static boolean[] projectedTopLevelColumns( + boolean readAll, List readColumnIds, int fieldCount) { + boolean[] projected = new boolean[fieldCount]; + if (readAll) { + Arrays.fill(projected, true); + return projected; + } + + if (readColumnIds == null || readColumnIds.isEmpty()) { + return projected; + } + + for (Integer id : readColumnIds) { + if (id != null && id >= 0 && id < fieldCount) { + projected[id] = true; + } + } + + return projected; + } + + private static void collectVariantColumns( + int rootColumnIndex, + org.apache.iceberg.types.Type icebergType, + Type parquetType, + int[] fieldPath, + String[] logicalPath, + String[] physicalPath, + List results, + Set nestedPaths) { + + switch (icebergType.typeId()) { + case VARIANT: + VariantColumnDescriptor variantColumn = + getOrCreateVariantColumn( + rootColumnIndex, parquetType, fieldPath, logicalPath, physicalPath, nestedPaths); + if (variantColumn != null) { + results.add(variantColumn); + } + return; + + case STRUCT: + collectFromStruct( + rootColumnIndex, + icebergType.asStructType(), + parquetType, + fieldPath, + logicalPath, + physicalPath, + results, + nestedPaths); + return; + + default: + // VARIANT shredding is not applied within arrays or maps, or other types + } + } + + private static void collectFromStruct( + int rootColumnIndex, + Types.StructType structType, + Type parquetType, + int[] fieldPath, + String[] logicalPath, + String[] physicalPath, + List results, + Set nestedPaths) { + + if (parquetType.isPrimitive()) { + return; + } + + GroupType parquetGroup = parquetType.asGroupType(); + List fields = structType.fields(); + + for (int i = 0; i < fields.size(); i++) { + Types.NestedField field = fields.get(i); + Type nestedParquetType = findParquetType(parquetGroup, field); + if (nestedParquetType == null) { + continue; + } + + collectVariantColumns( + rootColumnIndex, + field.type(), + nestedParquetType, + append(fieldPath, i), + append(logicalPath, field.name()), + append(physicalPath, nestedParquetType.getName()), + results, + nestedPaths); + } + } + + private static VariantColumnDescriptor getOrCreateVariantColumn( + int rootColumnIndex, + Type parquetType, + int[] fieldPath, + String[] logicalPath, + String[] physicalPath, + Set nestedPaths) { + if (parquetType.isPrimitive()) { + return null; + } + + GroupType variantGroup = parquetType.asGroupType(); + if (!variantGroup.containsField(VariantPathUtil.TYPED_VALUE)) { + // If there is no typed_value, variant is unshredded. Hive's reader is sufficient. + return null; + } + + Set projectedPaths = computeProjectedPaths(logicalPath, nestedPaths); + + // Prune the schema to match the projection so that the Reader only expects columns that will be present in the I/O + // We use the logical projection paths to prune the physical schema structure + Type prunedSchema = new VariantSchemaPruner(projectedPaths).prune(parquetType); + + return new VariantColumnDescriptor( + rootColumnIndex, fieldPath, physicalPath, prunedSchema); + } + + private static Set computeProjectedPaths(String[] logicalPath, Set nestedPaths) { + if (nestedPaths == null || nestedPaths.isEmpty()) { + return null; + } + + String prefix = String.join(".", logicalPath).toLowerCase() + "."; + Set projectedPaths = Sets.newHashSet(); + for (String path : nestedPaths) { + if (path.startsWith(prefix)) { + projectedPaths.add(path.substring(prefix.length())); + } + } + + if (projectedPaths.isEmpty()) { + // No paths matched this column in global nestedPaths: implies full column read. + // Set to null to enable fast path in computeRequestedColumns. + return null; + } + + return projectedPaths; + } + + private static int[] append(int[] path, int index) { + int[] copy = Arrays.copyOf(path, path.length + 1); + copy[path.length] = index; + return copy; + } + + private static String[] append(String[] path, String segment) { + String[] copy = Arrays.copyOf(path, path.length + 1); + copy[path.length] = segment; + return copy; + } + + private static List computeRequestedColumns( + MessageType fileSchema, List variantColumns) { + // Build the list of Parquet leaf columns needed to read all requested VARIANT columns. + List requested = Lists.newArrayList(); + for (ColumnDescriptor desc : fileSchema.getColumns()) { + String[] pathParts = desc.getPath(); + if (pathParts == null || pathParts.length == 0) { + continue; + } + + for (VariantColumnDescriptor vc : variantColumns) { + if (startsWith(pathParts, vc.physicalPath())) { + // Check if path exists in vc.prunedSchema() + if (containsPath(vc.prunedSchema(), pathParts, vc.physicalPath().length)) { + requested.add(desc); + } + break; + } + } + } + + return requested; + } + + private static boolean containsPath(Type root, String[] path, int offset) { + Type current = root; + for (int i = offset; i < path.length; i++) { + if (current.isPrimitive()) { + return false; + } + GroupType group = current.asGroupType(); + if (!group.containsField(path[i])) { + return false; + } + current = group.getType(path[i]); + } + return true; + } + + private static boolean startsWith(String[] path, String[] prefix) { + if (path == null || prefix == null || path.length < prefix.length) { + return false; + } + for (int i = 0; i < prefix.length; i++) { + if (!Objects.equals(path[i], prefix[i])) { + return false; + } + } + return true; + } + + /** + * Helper class to prune the Parquet physical schema of a Variant column based on projected paths. + * Handles the specific structure of Shredded Variants (metadata, value, typed_value). + */ + private static class VariantSchemaPruner { + private final ProjectionNode rootNode; + + VariantSchemaPruner(Set projectedPaths) { + this.rootNode = buildProjectionTree(projectedPaths); + } + + Type prune(Type rootType) { + if (rootNode == null) { + return rootType; + } + return prune(rootType, rootNode, true); + } + + private Type prune(Type type, ProjectionNode node, boolean isVariantWrapper) { + if (shouldStopPruning(type, node)) { + return type; + } + + GroupType group = type.asGroupType(); + List newFields = Lists.newArrayList(); + boolean changed = false; + + for (Type child : group.getFields()) { + Type prunedChild = null; + + if (isVariantWrapper) { + // Mode 1: Variant Wrapper (metadata, value, typed_value) + String name = child.getName(); + if (VariantPathUtil.METADATA.equals(name) || VariantPathUtil.VALUE.equals(name)) { + prunedChild = child; + } else if (VariantPathUtil.TYPED_VALUE.equals(name)) { + // Recurse into typed_value, switching to Logical Mode (isVariantWrapper = false) + // 'typed_value' is a container, so we pass the same 'node' down + prunedChild = prune(child, node, false); + } + } else { + // Mode 2: Logical Structure (structs, typed_value content) + ProjectionNode childNode = node.children.get(child.getName().toLowerCase()); + if (childNode != null) { + // Recurse, checking if the child itself is a nested Variant Wrapper + prunedChild = prune(child, childNode, isVariantWrapper(child)); + } + } + + if (isValid(prunedChild)) { + newFields.add(prunedChild); + if (prunedChild != child) { + changed = true; + } + } else { + changed = true; // Dropped + } + } + + if (!changed) { + return type; + } + + return group.withNewFields(newFields); + } + + private boolean shouldStopPruning(Type type, ProjectionNode node) { + if (node.isSelected || type.isPrimitive()) { + return true; + } + + OriginalType originalType = type.getOriginalType(); + return originalType == OriginalType.LIST || originalType == OriginalType.MAP; + } + + private boolean isVariantWrapper(Type type) { + if (type.isPrimitive()) { + return false; + } + return type.asGroupType().containsField(VariantPathUtil.TYPED_VALUE); + } + + private boolean isValid(Type child) { + return child != null && (child.isPrimitive() || !child.asGroupType().getFields().isEmpty()); + } + + private static ProjectionNode buildProjectionTree(Set paths) { + if (paths == null) { + return null; + } + + ProjectionNode root = new ProjectionNode(); + for (String path : paths) { + ProjectionNode current = root; + int start = 0; + int len = path.length(); + while (start < len) { + int dot = path.indexOf('.', start); + String segment = (dot == -1) ? path.substring(start) : path.substring(start, dot); + start = (dot == -1) ? len : dot + 1; + current = current.getOrCreate(segment.toLowerCase()); + } + current.isSelected = true; + } + return root; + } + + private static class ProjectionNode { + private final Map children = Maps.newHashMap(); + private boolean isSelected = false; + + ProjectionNode getOrCreate(String segment) { + return children.computeIfAbsent(segment, k -> new ProjectionNode()); + } + } + } +} diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/vector/HiveVectorizedReader.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/vector/HiveVectorizedReader.java index 20a73cba5d6d..e38184b2e6ae 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/vector/HiveVectorizedReader.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/vector/HiveVectorizedReader.java @@ -61,6 +61,7 @@ import org.apache.iceberg.parquet.ParquetFooterInputFromCache; import org.apache.iceberg.parquet.ParquetSchemaUtil; import org.apache.iceberg.parquet.TypeWithSchemaVisitor; +import org.apache.iceberg.parquet.VariantParquetFilters; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; @@ -163,7 +164,8 @@ public static CloseableIterable reader(Table table, Path path, case PARQUET: recordReader = parquetRecordReader(job, reporter, task, path, start, length, fileId, - getInitialColumnDefaults(table.schema().columns())); + getInitialColumnDefaults(table.schema().columns()), + HiveIcebergInputFormat.residualForReaderPruning(task, job)); break; default: throw new UnsupportedOperationException("Vectorized Hive reading unimplemented for format: " + format); @@ -232,7 +234,7 @@ private static RecordReader orcRecordReader(Jo private static RecordReader parquetRecordReader(JobConf job, Reporter reporter, FileScanTask task, Path path, long start, long length, SyntheticFileId fileId, - Map initialColumnDefaults) throws IOException { + Map initialColumnDefaults, Expression residual) throws IOException { InputSplit split = new FileSplit(path, start, length, job); VectorizedParquetInputFormat inputFormat = new VectorizedParquetInputFormat(); @@ -246,9 +248,11 @@ private static RecordReader parquetRecordReade ParquetMetadata parquetMetadata = footerData != null ? ParquetFileReader.readFooter(new ParquetFooterInputFromCache(footerData), ParquetMetadataConverter.NO_FILTER) : ParquetFileReader.readFooter(job, path); - inputFormat.setMetadata(parquetMetadata); - MessageType fileSchema = parquetMetadata.getFileMetaData().getSchema(); + ParquetMetadata prunedMetadata = + VariantParquetFilters.pruneVariantRowGroups(parquetMetadata, fileSchema, residual); + inputFormat.setMetadata(prunedMetadata); + MessageType typeWithIds = null; Schema expectedSchema = task.spec().schema(); @@ -264,7 +268,10 @@ private static RecordReader parquetRecordReade job.set(IOConstants.COLUMNS, psv.retrieveColumnNameList()); inputFormat.seInitialColumnDefaults(initialColumnDefaults); - return inputFormat.getRecordReader(split, job, reporter); + RecordReader reader = inputFormat.getRecordReader(split, job, reporter); + return ParquetVariantRecordReader + .tryWrap(reader, job, task, path, start, length, prunedMetadata) + .orElse(reader); } private static CloseableIterable createVectorizedRowBatchIterable( diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/vector/ParquetVariantRecordReader.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/vector/ParquetVariantRecordReader.java new file mode 100644 index 000000000000..566d262813b0 --- /dev/null +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/vector/ParquetVariantRecordReader.java @@ -0,0 +1,388 @@ +/* + * 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.iceberg.mr.hive.vector; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; +import java.util.function.ToIntFunction; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.ColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.StructColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; +import org.apache.hadoop.hive.ql.io.parquet.ParquetRecordReaderBase; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.RecordReader; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.mr.hive.variant.VariantProjectionUtil.VariantColumnDescriptor; +import org.apache.iceberg.mr.hive.variant.VariantProjectionUtil.VariantProjection; +import org.apache.iceberg.parquet.ParquetValueReader; +import org.apache.iceberg.parquet.ParquetVariantVisitor; +import org.apache.iceberg.parquet.VariantReaderBuilder; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.variants.Variant; +import org.apache.iceberg.variants.VariantMetadata; +import org.apache.iceberg.variants.VariantValue; +import org.apache.parquet.column.page.PageReadStore; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.Type; + +final class ParquetVariantRecordReader implements RecordReader { + + private final RecordReader delegate; + private final ParquetFileReader parquetReader; + + private final List variantColumns; + private final ParquetValueReader[] readers; + private final StructColumnVector[] structCvb; + + private long remainingInRowGroup; + + @SuppressWarnings("unchecked") + private ParquetVariantRecordReader( + RecordReader delegate, + JobConf job, + Path path, + ParquetMetadata parquetMetadata, + List blocks, + VariantProjection projection) throws IOException { + this.delegate = delegate; + this.variantColumns = projection.variantColumns(); + this.structCvb = new StructColumnVector[variantColumns.size()]; + this.readers = new ParquetValueReader[variantColumns.size()]; + + MessageType fileSchema = parquetMetadata.getFileMetaData().getSchema(); + for (int i = 0; i < variantColumns.size(); i++) { + readers[i] = createReader(fileSchema, variantColumns.get(i)); + } + + this.parquetReader = new ParquetFileReader( + job, parquetMetadata.getFileMetaData(), path, blocks, projection.requestedColumns()); + + advanceRowGroup(); + } + + /** + * Wrapper to read Iceberg shredded VARIANT data and populate Hive's VARIANT struct (metadata/value). + *

+ * Ensures correct reconstruction from shredded fields which standard Hive readers might miss. + */ + static Optional> tryWrap( + RecordReader delegate, + JobConf job, + FileScanTask task, + Path path, + long start, + long length, + ParquetMetadata parquetMetadata) throws IOException { + + if (delegate == null || parquetMetadata == null || parquetMetadata.getFileMetaData() == null) { + return Optional.empty(); + } + + MessageType fileSchema = parquetMetadata.getFileMetaData().getSchema(); + if (fileSchema == null) { + return Optional.empty(); + } + + VariantProjection projection = VariantProjection.create(fileSchema, job, task.spec().schema()); + if (projection == null) { + return Optional.empty(); + } + + List blocks = blocksForSplit(delegate, start, length, parquetMetadata); + if (blocks.isEmpty()) { + // Delegate reader will handle empty split; don't wrap. + return Optional.empty(); + } + + return Optional.of( + new ParquetVariantRecordReader( + delegate, job, path, parquetMetadata, blocks, projection)); + } + + private static List blocksForSplit( + RecordReader delegate, + long start, + long length, + ParquetMetadata parquetMetadata) { + + // If the underlying Hive Parquet reader already computed row-group filtering (e.g. from SARG), + // we must use the exact same blocks to keep this reader aligned with the delegate. + if (delegate instanceof ParquetRecordReaderBase parquetDelegate) { + List filteredBlocks = parquetDelegate.getFilteredBlocks(); + // Treat an empty list as authoritative (delegate filtered out all row groups). + if (filteredBlocks != null) { + return filteredBlocks; + } + } + // Fallback: compute blocks from split boundaries + List splitBlocks = Lists.newArrayList(); + for (BlockMetaData block : parquetMetadata.getBlocks()) { + long firstDataPage = block.getColumns().getFirst().getFirstDataPageOffset(); + if (firstDataPage >= start && firstDataPage < start + length) { + splitBlocks.add(block); + } + } + return splitBlocks; + } + + @Override + public boolean next(NullWritable key, VectorizedRowBatch value) throws IOException { + boolean hasNext = delegate.next(key, value); + if (!hasNext || value == null || value.size <= 0) { + return hasNext; + } + populateVariantStructColumns(value); + return true; + } + + private void populateVariantStructColumns(VectorizedRowBatch batch) throws IOException { + for (int i = 0; i < variantColumns.size(); i++) { + structCvb[i] = variantStructVector(batch, variantColumns.get(i)); + } + + int offset = 0; + while (offset < batch.size) { + ensureRowAvailable(); + if (remainingInRowGroup == 0) { + break; + } + int length = (int) Math.min(batch.size - offset, remainingInRowGroup); + processChunk(batch, offset, length); + + remainingInRowGroup -= length; + offset += length; + } + } + + private void processChunk(VectorizedRowBatch batch, int offset, int length) { + boolean selectedInUse = batch.selectedInUse; + int[] selected = batch.selected; + + for (int i = 0; i < variantColumns.size(); i++) { + StructColumnVector structVector = structCvb[i]; + ParquetValueReader reader = readers[i]; + + // variantStructVector guarantees fields != null and fields.length >= 2 when non-null + // and that metadata/value vectors are present. + BytesColumnVector metadataVector = (BytesColumnVector) structVector.fields[0]; + BytesColumnVector valueVector = (BytesColumnVector) structVector.fields[1]; + + ByteBuffer metadataBuffer = null; + byte[] lastMetadataArray = null; + + ByteBuffer valueBuffer = null; + byte[] lastValueArray = null; + + for (int pos = 0; pos < length; pos++) { + int batchIndex = offset + pos; + int row = selectedInUse ? selected[batchIndex] : batchIndex; + + Variant variant = reader.read(null); + + if (variant != null) { + structVector.isNull[row] = false; + metadataVector.isNull[row] = false; + valueVector.isNull[row] = false; + + metadataBuffer = writeMetadata( + metadataVector, row, variant.metadata(), metadataBuffer, lastMetadataArray); + lastMetadataArray = metadataVector.getValPreallocatedBytes(); + + valueBuffer = writeValue( + valueVector, row, variant.value(), valueBuffer, lastValueArray); + lastValueArray = valueVector.getValPreallocatedBytes(); + } else { + structVector.noNulls = false; + structVector.isNull[row] = true; + metadataVector.noNulls = false; + metadataVector.isNull[row] = true; + valueVector.noNulls = false; + valueVector.isNull[row] = true; + } + } + } + } + + @SuppressWarnings("checkstyle:CyclomaticComplexity") + private static StructColumnVector variantStructVector( + VectorizedRowBatch batch, VariantColumnDescriptor vc) throws IOException { + if (batch == null || vc == null || batch.cols == null) { + throw new IOException("Invalid batch or descriptor: batch=" + batch + ", vc=" + vc); + } + + int rootIndex = vc.rootColumnIndex(); + if (rootIndex < 0 || rootIndex >= batch.cols.length) { + throw new IOException("Root column index " + rootIndex + " out of bounds (cols=" + batch.cols.length + ")"); + } + + ColumnVector current = batch.cols[rootIndex]; + + for (int fieldIndex : vc.fieldPath()) { + if (!(current instanceof StructColumnVector struct)) { + throw new IOException("Expected nested StructColumnVector at field index " + fieldIndex + + " for column " + Arrays.toString(vc.physicalPath()) + ", found " + + (current == null ? "null" : current.getClass().getSimpleName())); + } + struct.isRepeating = false; + + if (struct.fields == null || fieldIndex < 0 || fieldIndex >= struct.fields.length) { + throw new IOException("Invalid field path index " + fieldIndex + " for struct with " + + (struct.fields == null ? "null" : struct.fields.length) + " fields"); + } + current = struct.fields[fieldIndex]; + } + + if (!(current instanceof StructColumnVector variantStruct)) { + throw new IOException("Expected Variant StructColumnVector at " + Arrays.toString(vc.physicalPath()) + + ", found " + (current == null ? "null" : current.getClass().getSimpleName())); + } + + if (variantStruct.fields == null || variantStruct.fields.length != 2) { + throw new IOException("Invalid Variant struct fields at " + Arrays.toString(vc.physicalPath()) + + ": expected exactly 2 fields (metadata, value), found " + + (variantStruct.fields == null ? "null" : variantStruct.fields.length)); + } + variantStruct.isRepeating = false; + + if (variantStruct.fields[0] == null) { + throw new IOException("Invalid Variant vector structure for column " + + Arrays.toString(vc.physicalPath()) + ": metadata vector is null."); + } + variantStruct.fields[0].isRepeating = false; + + if (variantStruct.fields[1] == null) { + throw new IOException("Invalid Variant vector structure for column " + + Arrays.toString(vc.physicalPath()) + ": value vector is null."); + } + variantStruct.fields[1].isRepeating = false; + + return variantStruct; + } + + private void ensureRowAvailable() throws IOException { + if (remainingInRowGroup <= 0) { + advanceRowGroup(); + } + } + + private void advanceRowGroup() throws IOException { + PageReadStore nextRowGroup = parquetReader.readNextRowGroup(); + if (nextRowGroup == null) { + remainingInRowGroup = 0; + return; + } + remainingInRowGroup = nextRowGroup.getRowCount(); + for (ParquetValueReader reader : readers) { + reader.setPageSource(nextRowGroup); + } + } + + @FunctionalInterface + private interface VariantWriteFunction { + void write(T data, ByteBuffer buffer, int offset); + } + + private static ByteBuffer writeMetadata( + BytesColumnVector vector, int row, VariantMetadata metadata, + ByteBuffer reusableBuffer, byte[] lastBackingArray) { + return writeToVector(vector, row, metadata, reusableBuffer, lastBackingArray, + VariantMetadata::sizeInBytes, VariantMetadata::writeTo); + } + + private static ByteBuffer writeValue( + BytesColumnVector vector, int row, VariantValue value, + ByteBuffer reusableBuffer, byte[] lastBackingArray) { + return writeToVector(vector, row, value, reusableBuffer, lastBackingArray, + VariantValue::sizeInBytes, VariantValue::writeTo); + } + + private static ByteBuffer writeToVector( + BytesColumnVector vector, int row, T data, + ByteBuffer reusableBuffer, byte[] lastBackingArray, + ToIntFunction sizeFunction, VariantWriteFunction writeFunction) { + if (vector == null || data == null) { + return reusableBuffer; + } + + int length = sizeFunction.applyAsInt(data); + vector.ensureValPreallocated(length); + + // Only create new ByteBuffer if backing array changed + byte[] currentBackingArray = vector.getValPreallocatedBytes(); + ByteBuffer resultBuffer = reusableBuffer; + if (resultBuffer == null || currentBackingArray != lastBackingArray) { + resultBuffer = ByteBuffer.wrap(currentBackingArray).order(ByteOrder.LITTLE_ENDIAN); + } + + writeFunction.write(data, resultBuffer, vector.getValPreallocatedStart()); + vector.setValPreallocated(row, length); + + return resultBuffer; + } + + @Override + public NullWritable createKey() { + return delegate.createKey(); + } + + @Override + public VectorizedRowBatch createValue() { + return delegate.createValue(); + } + + @Override + public long getPos() throws IOException { + return delegate.getPos(); + } + + @Override + public void close() throws IOException { + try { + delegate.close(); + } finally { + parquetReader.close(); + } + } + + @Override + public float getProgress() throws IOException { + return delegate.getProgress(); + } + + private static ParquetValueReader createReader( + MessageType fileSchema, VariantColumnDescriptor variantColumn) { + Type readSchema = variantColumn.prunedSchema(); + + // Prune the reader schema to match the projection so we don't expect columns skipped by I/O filtering + return (ParquetValueReader) ParquetVariantVisitor.visit( + readSchema.asGroupType(), + new VariantReaderBuilder(fileSchema, Arrays.asList(variantColumn.physicalPath()))); + } +} diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveFileWriterFactory.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveFileWriterFactory.java index 049a5a0dfb49..234cf928432b 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveFileWriterFactory.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveFileWriterFactory.java @@ -86,7 +86,7 @@ protected void configurePositionDelete(Avro.DeleteWriteBuilder builder) { protected void configureDataWrite(Parquet.DataWriteBuilder builder) { builder.createWriterFunc(GenericParquetWriter::create); // Configure variant shredding if enabled and a sample record is available - if (VariantUtil.shouldUseVariantShredding(properties::get, dataSchema())) { + if (VariantUtil.shouldUseVariantShredding(properties, dataSchema())) { setVariantShreddingFunc(builder, VariantUtil.variantShreddingFunc(sampleRecord, dataSchema())); } } diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergCopyOnWriteRecordWriter.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergCopyOnWriteRecordWriter.java index f48bf21345e2..b2a7d7c29c90 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergCopyOnWriteRecordWriter.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergCopyOnWriteRecordWriter.java @@ -36,7 +36,7 @@ import org.apache.iceberg.mr.mapred.Container; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -class HiveIcebergCopyOnWriteRecordWriter extends SchemaInferringDefaultsWriter { +class HiveIcebergCopyOnWriteRecordWriter extends HiveIcebergDefaultValuesWriter { private final GenericRecord rowDataTemplate; private final List replacedDataFiles; @@ -66,7 +66,7 @@ public void write(Writable row) throws IOException { .build(); replacedDataFiles.add(dataFile); } else { - writeOrBuffer(rowData); + write(rowData); } } diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergDefaultValuesWriter.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergDefaultValuesWriter.java new file mode 100644 index 000000000000..5651895275fe --- /dev/null +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergDefaultValuesWriter.java @@ -0,0 +1,94 @@ +/* + * 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.iceberg.mr.hive.writer; + +import java.io.IOException; +import java.util.List; +import java.util.Set; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.hive.HiveSchemaUtil; +import org.apache.iceberg.io.DataWriteResult; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.io.PartitioningWriter; +import org.apache.iceberg.mr.hive.writer.WriterBuilder.Context; +import org.apache.iceberg.types.Types.NestedField; + +/** + * Base writer that applies Iceberg/Hive schema defaulting for missing columns and struct fields. + * + *

For Parquet writes with variant shredding enabled, the underlying data writer may be wrapped to buffer a small + * sample of records to initialize VARIANT {@code typed_value} schema inference before the first Parquet file is + * created. + */ +abstract class HiveIcebergDefaultValuesWriter extends HiveIcebergWriterBase { + + private final int currentSpecId; + private final Set missingColumns; + private final List missingOrStructFields; + + HiveIcebergDefaultValuesWriter( + Table table, + HiveFileWriterFactory fileWriterFactory, + OutputFileFactory dataFileFactory, + Context context) { + super(table, newDataWriterWithVariantSchemaInference(table, fileWriterFactory, dataFileFactory, context)); + + Schema schema = table.schema(); + this.currentSpecId = table.spec().specId(); + this.missingColumns = + context != null && context.missingColumns() != null ? context.missingColumns() : Set.of(); + + this.missingOrStructFields = + schema.columns().stream() + .filter(field -> missingColumns.contains(field.name()) || field.type().isStructType()) + .toList(); + } + + private static PartitioningWriter newDataWriterWithVariantSchemaInference( + Table table, + HiveFileWriterFactory fileWriterFactory, + OutputFileFactory dataFileFactory, + Context context) { + PartitioningWriter delegate = + newDataWriter(table, fileWriterFactory, dataFileFactory, context); + return ParquetVariantRecordWriter + .tryWrap(table, fileWriterFactory, context, delegate) + .orElse(delegate); + } + + protected final void applyDefaultValues(Record record) { + HiveSchemaUtil.setDefaultValues(record, missingOrStructFields, missingColumns); + } + + protected final void write(Record record) { + applyDefaultValues(record); + writer.write(record, specs.get(currentSpecId), partition(record, currentSpecId)); + } + + @Override + public void close(boolean abort) throws IOException { + if (abort && writer instanceof ParquetVariantRecordWriter variantWriter) { + variantWriter.abort(); + } + super.close(abort); + } +} diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergRecordWriter.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergRecordWriter.java index cb9a21e9780c..ee739f973109 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergRecordWriter.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergRecordWriter.java @@ -31,7 +31,7 @@ import org.apache.iceberg.mr.hive.writer.WriterBuilder.Context; import org.apache.iceberg.mr.mapred.Container; -class HiveIcebergRecordWriter extends SchemaInferringDefaultsWriter { +class HiveIcebergRecordWriter extends HiveIcebergDefaultValuesWriter { HiveIcebergRecordWriter(Table table, HiveFileWriterFactory fileWriterFactory, OutputFileFactory dataFileFactory, Context context) { @@ -41,7 +41,7 @@ class HiveIcebergRecordWriter extends SchemaInferringDefaultsWriter { @Override public void write(Writable row) throws IOException { Record record = ((Container) row).get(); - writeOrBuffer(record); + write(record); } @Override diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/SchemaInferringDefaultsWriter.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/ParquetVariantRecordWriter.java similarity index 51% rename from iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/SchemaInferringDefaultsWriter.java rename to iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/ParquetVariantRecordWriter.java index 487085b6631d..7168157b98d4 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/SchemaInferringDefaultsWriter.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/ParquetVariantRecordWriter.java @@ -22,87 +22,135 @@ import java.io.IOException; import java.util.BitSet; import java.util.List; -import java.util.Set; +import java.util.Optional; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.hive.HiveSchemaUtil; -import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.io.DataWriteResult; +import org.apache.iceberg.io.PartitioningWriter; import org.apache.iceberg.mr.hive.writer.WriterBuilder.Context; import org.apache.iceberg.parquet.VariantUtil; import org.apache.iceberg.parquet.VariantUtil.VariantField; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Types.NestedField; +import org.apache.iceberg.util.StructLikeUtil; -abstract class SchemaInferringDefaultsWriter extends HiveIcebergWriterBase { +/** + * Writer-side helper that buffers a small sample of records to initialize Parquet VARIANT shredding schema inference. + * + *

This is only applicable for Parquet writes when {@code variant.shredding.enabled=true}. When enabled, Iceberg + * may use a data-driven schema for {@code typed_value}. To avoid picking the first record arbitrarily (or + * initializing before any variant value is seen), this buffers up to a small number of records and initializes the + * {@link HiveFileWriterFactory} with an accumulated sample record once enough variant fields have been observed. + */ +final class ParquetVariantRecordWriter implements PartitioningWriter { private static final int VARIANT_SAMPLE_BUFFER_SIZE = 100; + private final PartitioningWriter delegate; private final HiveFileWriterFactory fileWriterFactory; - private final int currentSpecId; - private final Set missingColumns; - private final List missingOrStructFields; - private final List variantFields; private final BitSet sampledVariantFields; + private boolean sampleInitialized = false; - private final List buffer; + private final List buffer; private final Record accumulatedSample; - private boolean sampleInitialized = false; - SchemaInferringDefaultsWriter( + private record BufferedWrite(Record record, PartitionSpec spec, StructLike partition) { + } + + static Optional> tryWrap( Table table, HiveFileWriterFactory fileWriterFactory, - OutputFileFactory dataFileFactory, - Context context) { + Context context, + PartitioningWriter delegate) { + if (table == null || fileWriterFactory == null || context == null || delegate == null) { + return Optional.empty(); + } + + if (context.dataFileFormat() != FileFormat.PARQUET) { + return Optional.empty(); + } - super(table, newDataWriter(table, fileWriterFactory, dataFileFactory, context)); Schema schema = table.schema(); - this.fileWriterFactory = fileWriterFactory; + List variantFields = VariantUtil.variantFieldsForShredding(table.properties(), schema); + if (variantFields.isEmpty()) { + return Optional.empty(); + } - this.currentSpecId = table.spec().specId(); - this.missingColumns = context.missingColumns(); - this.missingOrStructFields = schema.columns().stream() - .filter(field -> missingColumns.contains(field.name()) || field.type().isStructType()) - .toList(); + return Optional.of(new ParquetVariantRecordWriter(delegate, fileWriterFactory, schema, variantFields)); + } - this.variantFields = VariantUtil.variantFieldsForShredding(table.properties(), schema); + private ParquetVariantRecordWriter( + PartitioningWriter delegate, + HiveFileWriterFactory fileWriterFactory, + Schema schema, + List variantFields) { + this.delegate = delegate; + this.fileWriterFactory = fileWriterFactory; + this.variantFields = variantFields; this.sampledVariantFields = new BitSet(variantFields.size()); + this.buffer = Lists.newArrayListWithCapacity(VARIANT_SAMPLE_BUFFER_SIZE); + this.accumulatedSample = GenericRecord.create(schema); + } - boolean shouldBuffer = !variantFields.isEmpty(); - this.buffer = shouldBuffer ? Lists.newArrayListWithCapacity(VARIANT_SAMPLE_BUFFER_SIZE) : null; - this.accumulatedSample = shouldBuffer ? GenericRecord.create(schema) : null; + void abort() { + buffer.clear(); } - protected void writeOrBuffer(Record record) { - HiveSchemaUtil.setDefaultValues(record, missingOrStructFields, missingColumns); + @Override + public void write(Record record, PartitionSpec spec, StructLike partition) { + if (record == null) { + delegate.write(null, spec, partition); + return; + } - if (buffer != null && !sampleInitialized) { + if (!sampleInitialized) { accumulateSample(record); if (allVariantFieldsSampled() || buffer.size() >= VARIANT_SAMPLE_BUFFER_SIZE) { - // Use accumulated sample for schema inference fileWriterFactory.initialize(accumulatedSample); sampleInitialized = true; - - flushBufferedRecords(); + flush(); + delegate.write(record, spec, partition); } else { - buffer.add(record.copy()); - return; + buffer.add(new BufferedWrite(record.copy(), spec, StructLikeUtil.copy(partition))); } + return; } - writeRecord(record); + + delegate.write(record, spec, partition); } - private void writeRecord(Record record) { - writer.write(record, specs.get(currentSpecId), partition(record, currentSpecId)); + @Override + public void close() throws IOException { + try { + if (!buffer.isEmpty()) { + if (!sampleInitialized) { + // Initialize using whatever we have accumulated so far. + fileWriterFactory.initialize(accumulatedSample); + sampleInitialized = true; + } + flush(); + } + } finally { + delegate.close(); + } } - private void flushBufferedRecords() { - for (Record bufferedRecord : buffer) { - writeRecord(bufferedRecord); + @Override + public DataWriteResult result() { + return delegate.result(); + } + + private void flush() { + for (BufferedWrite buffered : buffer) { + delegate.write(buffered.record(), buffered.spec(), buffered.partition()); } buffer.clear(); } @@ -112,7 +160,7 @@ private boolean allVariantFieldsSampled() { } private void accumulateSample(Record record) { - if (accumulatedSample == null || allVariantFieldsSampled()) { + if (allVariantFieldsSampled()) { return; } for (int fieldIndex = sampledVariantFields.nextClearBit(0); @@ -128,6 +176,7 @@ private void trySampleVariantField(int fieldIndex, Record record) { if (!VariantUtil.isShreddable(val)) { return; } + HiveSchemaUtil.setStructField(accumulatedSample, variantField.path(), val); sampledVariantFields.set(fieldIndex); } @@ -141,21 +190,4 @@ private static Object safeGet(VariantField variantField, Record record) { } } - @Override - public void close(boolean abort) throws IOException { - if (buffer != null) { - if (abort) { - // Don't write anything on abort. Just drop any buffered records. - buffer.clear(); - } else if (!buffer.isEmpty()) { - if (!sampleInitialized) { - // Use whatever we have accumulated so far - fileWriterFactory.initialize(accumulatedSample); - } - flushBufferedRecords(); - } - } - super.close(abort); - } - } diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergRecordReader.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergRecordReader.java index f86072a39cd4..b24bc912f0d7 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergRecordReader.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergRecordReader.java @@ -192,7 +192,9 @@ private CloseableIterable openGeneric(FileScanTask task, Schema readSchema) { CloseableIterable iterable = switch (file.format()) { case AVRO -> newAvroIterable(inputFile, task, readSchema); case ORC -> newOrcIterable(inputFile, task, residual, readSchema); - case PARQUET -> newParquetIterable(inputFile, task, residual, readSchema); + case PARQUET -> newParquetIterable(inputFile, task, + HiveIcebergInputFormat.residualForReaderPruning(task, getContext().getConfiguration()), + readSchema); default -> throw new UnsupportedOperationException( String.format("Cannot read %s file: %s", file.format().name(), file.location())); }; diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/parquet/ReadConf.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/parquet/ReadConf.java new file mode 100644 index 000000000000..c350a3d7d562 --- /dev/null +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/parquet/ReadConf.java @@ -0,0 +1,226 @@ +/* + * 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.iceberg.parquet; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.iceberg.Schema; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.parquet.ParquetReadOptions; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ColumnPath; +import org.apache.parquet.schema.MessageType; + +/** + * Configuration for Parquet readers. + * + * @param type of value to read + */ +class ReadConf { + private final ParquetFileReader reader; + private final InputFile file; + private final ParquetReadOptions options; + private final MessageType projection; + private final ParquetValueReader model; + private final VectorizedReader vectorizedModel; + private final List rowGroups; + private final boolean[] shouldSkip; + private final long totalValues; + private final boolean reuseContainers; + private final Integer batchSize; + + // List of column chunk metadata for each row group + private final List> columnChunkMetaDataForRowGroups; + + @SuppressWarnings("unchecked") + ReadConf( + InputFile file, + ParquetReadOptions options, + Schema expectedSchema, + Expression filter, + Function> readerFunc, + Function> batchedReaderFunc, + NameMapping nameMapping, + boolean reuseContainers, + boolean caseSensitive, + Integer bSize) { + this.file = file; + this.options = options; + this.reader = newReader(file, options); + MessageType fileSchema = reader.getFileMetaData().getSchema(); + + MessageType typeWithIds; + if (ParquetSchemaUtil.hasIds(fileSchema)) { + typeWithIds = fileSchema; + this.projection = ParquetSchemaUtil.pruneColumns(fileSchema, expectedSchema); + } else if (nameMapping != null) { + typeWithIds = ParquetSchemaUtil.applyNameMapping(fileSchema, nameMapping); + this.projection = ParquetSchemaUtil.pruneColumns(typeWithIds, expectedSchema); + } else { + typeWithIds = ParquetSchemaUtil.addFallbackIds(fileSchema); + this.projection = ParquetSchemaUtil.pruneColumnsFallback(fileSchema, expectedSchema); + } + + this.rowGroups = reader.getRowGroups(); + this.shouldSkip = new boolean[rowGroups.size()]; + + ParquetMetricsRowGroupFilter statsFilter = null; + ParquetDictionaryRowGroupFilter dictFilter = null; + ParquetBloomRowGroupFilter bloomFilter = null; + if (filter != null) { + statsFilter = new ParquetMetricsRowGroupFilter(expectedSchema, filter, caseSensitive); + dictFilter = new ParquetDictionaryRowGroupFilter(expectedSchema, filter, caseSensitive); + bloomFilter = new ParquetBloomRowGroupFilter(expectedSchema, filter, caseSensitive); + } + + boolean[] variantRowGroupMayMatch = + VariantParquetFilters.variantRowGroupMayMatch(fileSchema, filter, rowGroups); + + long computedTotalValues = 0L; + for (int i = 0; i < shouldSkip.length; i += 1) { + BlockMetaData rowGroup = rowGroups.get(i); + boolean shouldRead = + (variantRowGroupMayMatch == null || variantRowGroupMayMatch[i]) && + (filter == null || + statsFilter.shouldRead(typeWithIds, rowGroup) && + dictFilter.shouldRead(typeWithIds, rowGroup, reader.getDictionaryReader(rowGroup)) && + bloomFilter.shouldRead(typeWithIds, rowGroup, reader.getBloomFilterDataReader(rowGroup))); + this.shouldSkip[i] = !shouldRead; + if (shouldRead) { + computedTotalValues += rowGroup.getRowCount(); + } + } + + this.totalValues = computedTotalValues; + if (readerFunc != null) { + this.model = (ParquetValueReader) readerFunc.apply(typeWithIds); + this.vectorizedModel = null; + this.columnChunkMetaDataForRowGroups = null; + } else { + this.model = null; + this.vectorizedModel = (VectorizedReader) batchedReaderFunc.apply(typeWithIds); + this.columnChunkMetaDataForRowGroups = getColumnChunkMetadataForRowGroups(); + } + + this.reuseContainers = reuseContainers; + this.batchSize = bSize; + } + + private ReadConf(ReadConf toCopy) { + this.reader = null; + this.file = toCopy.file; + this.options = toCopy.options; + this.projection = toCopy.projection; + this.model = toCopy.model; + this.rowGroups = toCopy.rowGroups; + this.shouldSkip = toCopy.shouldSkip; + this.totalValues = toCopy.totalValues; + this.reuseContainers = toCopy.reuseContainers; + this.batchSize = toCopy.batchSize; + this.vectorizedModel = toCopy.vectorizedModel; + this.columnChunkMetaDataForRowGroups = toCopy.columnChunkMetaDataForRowGroups; + } + + ParquetFileReader reader() { + if (reader != null) { + reader.setRequestedSchema(projection); + return reader; + } + + ParquetFileReader newReader = newReader(file, options); + newReader.setRequestedSchema(projection); + return newReader; + } + + ParquetValueReader model() { + return model; + } + + VectorizedReader vectorizedModel() { + return vectorizedModel; + } + + boolean[] shouldSkip() { + return shouldSkip; + } + + long totalValues() { + return totalValues; + } + + boolean reuseContainers() { + return reuseContainers; + } + + Integer batchSize() { + return batchSize; + } + + List> columnChunkMetadataForRowGroups() { + return columnChunkMetaDataForRowGroups; + } + + ReadConf copy() { + return new ReadConf<>(this); + } + + private static ParquetFileReader newReader(InputFile file, ParquetReadOptions options) { + try { + return ParquetFileReader.open(ParquetIO.file(file), options); + } catch (IOException e) { + throw new RuntimeIOException(e, "Failed to open Parquet file: %s", file.location()); + } + } + + private List> getColumnChunkMetadataForRowGroups() { + Set projectedColumns = + projection.getColumns().stream() + .map(columnDescriptor -> ColumnPath.get(columnDescriptor.getPath())) + .collect(Collectors.toSet()); + ImmutableList.Builder> listBuilder = + ImmutableList.builder(); + for (int i = 0; i < rowGroups.size(); i++) { + if (!shouldSkip[i]) { + BlockMetaData blockMetaData = rowGroups.get(i); + ImmutableMap.Builder mapBuilder = ImmutableMap.builder(); + blockMetaData.getColumns().stream() + .filter(columnChunkMetaData -> projectedColumns.contains(columnChunkMetaData.getPath())) + .forEach( + columnChunkMetaData -> + mapBuilder.put(columnChunkMetaData.getPath(), columnChunkMetaData)); + listBuilder.add(mapBuilder.build()); + } else { + listBuilder.add(ImmutableMap.of()); + } + } + return listBuilder.build(); + } +} diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/parquet/VariantParquetFilters.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/parquet/VariantParquetFilters.java new file mode 100644 index 000000000000..32149e28314f --- /dev/null +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/parquet/VariantParquetFilters.java @@ -0,0 +1,579 @@ +/* + * 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.iceberg.parquet; + +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.IdentityHashMap; +import java.util.List; +import java.util.Set; +import org.apache.commons.collections4.CollectionUtils; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expression.Operation; +import org.apache.iceberg.expressions.ExpressionVisitors; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.expressions.UnboundPredicate; +import org.apache.iceberg.mr.hive.variant.VariantPathUtil; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.parquet.filter2.compat.FilterCompat; +import org.apache.parquet.filter2.compat.RowGroupFilter; +import org.apache.parquet.filter2.predicate.FilterApi; +import org.apache.parquet.filter2.predicate.FilterPredicate; +import org.apache.parquet.filter2.predicate.Operators; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ColumnPath; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.LogicalTypeAnnotation.StringLogicalTypeAnnotation; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; + +/** + * Parquet row-group pruning for Iceberg {@code VARIANT} predicates when shredding is enabled. + * + *

Hive can produce Iceberg expressions that reference shredded VARIANT pseudo-columns (for example + * {@code payload.typed_value.tier}) or use {@code extract(payload, "$.tier", "string")}. + * + *

This class converts those predicates into Parquet filter2 predicates against the physical Parquet columns + * (for example {@code payload.typed_value.tier.typed_value}) and uses {@link RowGroupFilter} to evaluate row groups. + * + *

Correctness: pruning must not introduce false negatives. Because shredding can fall back to the + * serialized {@code ...value} column, a row group rejected by the Parquet predicate is only dropped when all relevant + * fallback {@code ...value} columns are provably all-null for that row group. + * + *

This is an optimization only; Hive still evaluates the full predicate after reading. + */ +public final class VariantParquetFilters { + + private VariantParquetFilters() { + } + + static FilterCompat.Filter toParquetFilter(MessageType schema, Expression expression) { + ResolvedVariantFilter resolved = resolveVariantFilter(schema, expression); + return resolved == null ? null : FilterCompat.get(resolved.predicate()); + } + + /** Single predicate built from one variant field. */ + private record ResolvedVariantPredicate(FilterPredicate predicate, ColumnPath fallbackValueColumn) { + } + + /** Aggregated filter built from entire expression tree. */ + private record ResolvedVariantFilter(FilterPredicate predicate, List fallbackValueColumns) { + } + + private static ResolvedVariantFilter resolveVariantFilter(MessageType schema, Expression expression) { + if (schema == null || expression == null) { + return null; + } + + VariantPredicateToParquetVisitor visitor = new VariantPredicateToParquetVisitor(schema); + FilterPredicate predicate = ExpressionVisitors.visit(expression, visitor); + if (predicate == null) { + return null; + } + return new ResolvedVariantFilter(predicate, visitor.fallbackValueColumns()); + } + + public static boolean[] variantRowGroupMayMatch( + MessageType fileSchema, Expression filter, List rowGroups) { + if (fileSchema == null || filter == null || rowGroups == null || rowGroups.isEmpty()) { + return null; + } + + ResolvedVariantFilter resolved = resolveVariantFilter(fileSchema, filter); + if (resolved == null) { + return null; + } + + FilterCompat.Filter parquetFilter = FilterCompat.get(resolved.predicate()); + List matchingRowGroups = RowGroupFilter.filterRowGroups(parquetFilter, rowGroups, fileSchema); + Set matchingSet = Collections.newSetFromMap(new IdentityHashMap<>()); + matchingSet.addAll(matchingRowGroups); + + Set fallbackValueColumns = Sets.newHashSet(resolved.fallbackValueColumns()); + boolean[] mayMatch = new boolean[rowGroups.size()]; + + for (int i = 0; i < rowGroups.size(); i++) { + BlockMetaData rowGroup = rowGroups.get(i); + mayMatch[i] = matchingSet.contains(rowGroup) || mayMatchViaFallback(rowGroup, fallbackValueColumns); + } + + return mayMatch; + } + + private static boolean mayMatchViaFallback(BlockMetaData rowGroup, Set fallbackValueColumns) { + if (fallbackValueColumns.isEmpty()) { + return true; + } + + for (ColumnChunkMetaData col : rowGroup.getColumns()) { + if (fallbackValueColumns.contains(col.getPath()) && !isColumnAllNull(col)) { + return true; + } + } + return false; + } + + private static List pruneVariantRowGroups( + MessageType fileSchema, Expression filter, List rowGroups) { + boolean[] mayMatch = variantRowGroupMayMatch(fileSchema, filter, rowGroups); + if (mayMatch == null) { + return rowGroups; + } + + List kept = Lists.newArrayListWithCapacity(rowGroups.size()); + for (int i = 0; i < rowGroups.size(); i++) { + if (mayMatch[i]) { + kept.add(rowGroups.get(i)); + } + } + + return kept.size() == rowGroups.size() ? rowGroups : kept; + } + + /** Returns Parquet metadata with row groups pruned using best-effort VARIANT pruning. */ + public static ParquetMetadata pruneVariantRowGroups( + ParquetMetadata parquetMetadata, MessageType fileSchema, Expression filter) { + if (parquetMetadata == null || filter == null) { + return parquetMetadata; + } + + List rowGroups = parquetMetadata.getBlocks(); + if (rowGroups == null || rowGroups.isEmpty()) { + return parquetMetadata; + } + + MessageType schema = fileSchema; + if (schema == null) { + if (parquetMetadata.getFileMetaData() == null) { + return parquetMetadata; + } + schema = parquetMetadata.getFileMetaData().getSchema(); + } + if (schema == null) { + return parquetMetadata; + } + + List kept = pruneVariantRowGroups(schema, filter, rowGroups); + if (kept == rowGroups || parquetMetadata.getFileMetaData() == null) { + return parquetMetadata; + } + + return new ParquetMetadata(parquetMetadata.getFileMetaData(), kept); + } + + private static boolean isColumnAllNull(ColumnChunkMetaData meta) { + if (meta == null || meta.getStatistics() == null || !meta.getStatistics().isNumNullsSet()) { + return false; + } + + return meta.getStatistics().getNumNulls() == meta.getValueCount(); + } + + private static final class VariantPredicateToParquetVisitor + extends ExpressionVisitors.ExpressionVisitor { + private final MessageType schema; + private final List fallbackValueColumns = Lists.newArrayList(); + + private VariantPredicateToParquetVisitor(MessageType schema) { + this.schema = schema; + } + + List fallbackValueColumns() { + return fallbackValueColumns; + } + + @Override + public FilterPredicate alwaysTrue() { + return null; + } + + @Override + public FilterPredicate alwaysFalse() { + return null; + } + + @Override + public FilterPredicate not(FilterPredicate child) { + return child == null ? null : FilterApi.not(child); + } + + @Override + public FilterPredicate and(FilterPredicate left, FilterPredicate right) { + // For AND, partial pushdown is safe: unconvertible predicates are evaluated by Hive after reading. + if (left == null) { + return right; + } else if (right == null) { + return left; + } + return FilterApi.and(left, right); + } + + @Override + public FilterPredicate or(FilterPredicate left, FilterPredicate right) { + // For OR, dropping an unconvertible side is unsafe: it may filter out rows that should match. + if (left == null || right == null) { + return null; + } + return FilterApi.or(left, right); + } + + @Override + public FilterPredicate predicate(UnboundPredicate predicate) { + if (!isSupportedOperation(predicate.op())) { + return null; + } + + VariantColumnPath columnPath = extractVariantColumnPath(predicate); + if (columnPath == null) { + return null; + } + + // Handle unary predicates (IS_NULL, NOT_NULL) + if (predicate.op() == Operation.IS_NULL || + predicate.op() == Operation.NOT_NULL) { + ResolvedVariantPredicate resolved = resolveVariantNullPredicate( + schema, columnPath.variantPath, columnPath.typedValuePath, predicate.op()); + if (resolved == null) { + return null; + } + fallbackValueColumns.add(resolved.fallbackValueColumn()); + return resolved.predicate(); + } + + // Handle literal predicates (EQ, NOT_EQ, LT, etc.) + Literal literal = predicate.literal(); + if (literal == null || literal.value() == null) { + return null; + } + + ResolvedVariantPredicate resolved = resolveVariantValuePredicate( + schema, columnPath.variantPath, columnPath.typedValuePath, predicate.op(), literal.value()); + if (resolved == null) { + return null; + } + + fallbackValueColumns.add(resolved.fallbackValueColumn()); + return resolved.predicate(); + } + + private static boolean isSupportedOperation(Operation op) { + return switch (op) { + case EQ, NOT_EQ, LT, LT_EQ, GT, GT_EQ, IS_NULL, NOT_NULL -> true; + default -> false; + }; + } + + private record VariantColumnPath(List variantPath, List typedValuePath) { + } + + private static VariantColumnPath extractVariantColumnPath(UnboundPredicate predicate) { + String shreddedColumnPath = VariantPathUtil.extractVariantShreddedColumn(predicate); + if (shreddedColumnPath == null) { + return null; + } + + int typedValueIdx = shreddedColumnPath.indexOf(VariantPathUtil.TYPED_VALUE_SEGMENT); + if (typedValueIdx < 0) { + return null; + } + + String variantPathStr = shreddedColumnPath.substring(0, typedValueIdx); + String typedValuePathStr = + shreddedColumnPath.substring(typedValueIdx + VariantPathUtil.TYPED_VALUE_SEGMENT.length()); + + List variantPath = VariantPathUtil.splitPath(variantPathStr); + List typedValuePath = typedValuePathStr.isEmpty() ? + Collections.emptyList() : VariantPathUtil.splitPath(typedValuePathStr); + + return new VariantColumnPath(variantPath, typedValuePath); + } + } + + private static ResolvedVariantPredicate resolveVariantNullPredicate( + MessageType schema, List variantPath, List typedValuePath, Operation op) { + return resolveVariantPredicate(schema, variantPath, typedValuePath, op, null); + } + + private static ResolvedVariantPredicate resolveVariantValuePredicate( + MessageType schema, List variantPath, List typedValuePath, + Operation op, Object literalValue) { + return resolveVariantPredicate(schema, variantPath, typedValuePath, op, literalValue); + } + + private static ResolvedVariantPredicate resolveVariantPredicate( + MessageType schema, List variantPath, List typedValuePath, + Operation op, Object literalValue) { + + if (CollectionUtils.isEmpty(variantPath) || CollectionUtils.isEmpty(typedValuePath)) { + return null; + } + GroupType variantGroup = resolveVariantGroup(schema, variantPath); + if (variantGroup == null) { + return null; + } + GroupType currentGroup = rootTypedValueGroup(variantGroup); + if (currentGroup == null) { + return null; + } + + List columnPathSegments = + Lists.newArrayListWithCapacity(variantPath.size() + typedValuePath.size() + 2); + columnPathSegments.addAll(variantPath); + columnPathSegments.add(ParquetVariantVisitor.TYPED_VALUE); + + for (int i = 0; i < typedValuePath.size(); i++) { + String fieldName = typedValuePath.get(i); + GroupType objectFieldGroup = resolveObjectFieldGroup(currentGroup, fieldName); + if (objectFieldGroup == null) { + return null; + } + + columnPathSegments.add(fieldName); + + boolean last = i == typedValuePath.size() - 1; + if (last) { + PrimitiveType leaf = leafTypedValuePrimitive(objectFieldGroup); + if (leaf == null) { + return null; + } + columnPathSegments.add(ParquetVariantVisitor.TYPED_VALUE); + + // Dispatch to appropriate builder based on whether we have a literal value + if (literalValue == null) { + return buildNullPredicate(columnPathSegments, op); + } else { + return buildValuePredicate(columnPathSegments, leaf, op, literalValue); + } + } + + currentGroup = objectFieldGroup; + } + + return null; + } + + private static GroupType resolveVariantGroup(MessageType schema, List variantPath) { + try { + Type variantType = schema.getType(variantPath.toArray(new String[0])); + if (variantType == null || variantType.isPrimitive()) { + return null; + } + return variantType.asGroupType(); + } catch (RuntimeException e) { + return null; + } + } + + private static GroupType rootTypedValueGroup(GroupType variantGroup) { + Type rootTypedValue = ParquetSchemaUtil.fieldType(variantGroup, ParquetVariantVisitor.TYPED_VALUE); + // Root typed_value is not an object; cannot resolve an object path. + return rootTypedValue != null && !rootTypedValue.isPrimitive() ? rootTypedValue.asGroupType() : null; + } + + private static GroupType resolveObjectFieldGroup(GroupType parent, String fieldName) { + Type fieldType = ParquetSchemaUtil.fieldType(parent, fieldName); + return fieldType != null && !fieldType.isPrimitive() ? fieldType.asGroupType() : null; + } + + private static PrimitiveType leafTypedValuePrimitive(GroupType objectFieldGroup) { + Type leaf = ParquetSchemaUtil.fieldType(objectFieldGroup, ParquetVariantVisitor.TYPED_VALUE); + return leaf != null && leaf.isPrimitive() ? leaf.asPrimitiveType() : null; + } + + private static ResolvedVariantPredicate buildNullPredicate( + List typedValueColumnPathSegments, Operation op) { + String typedValueColumnPath = String.join(".", typedValueColumnPathSegments); + + FilterPredicate predicate = (op == Operation.IS_NULL) ? + FilterApi.eq( + FilterApi.binaryColumn(typedValueColumnPath), + null) : + FilterApi.notEq( + FilterApi.binaryColumn(typedValueColumnPath), + null); + + return toResolvedPredicate(typedValueColumnPathSegments, predicate); + } + + private static ResolvedVariantPredicate buildValuePredicate( + List typedValueColumnPathSegments, PrimitiveType typedValueLeaf, + Operation op, Object literalValue) { + String typedValueColumnPath = String.join(".", typedValueColumnPathSegments); + + FilterPredicate predicate = PrimitivePredicateFactory.toParquetPredicate( + typedValueLeaf, typedValueColumnPath, op, literalValue); + + return toResolvedPredicate(typedValueColumnPathSegments, predicate); + } + + private static ResolvedVariantPredicate toResolvedPredicate( + List typedValueColumnPathSegments, FilterPredicate predicate) { + if (predicate == null) { + return null; + } + String[] fallbackValuePathSegments = typedValueColumnPathSegments.toArray(new String[0]); + fallbackValuePathSegments[fallbackValuePathSegments.length - 1] = ParquetVariantVisitor.VALUE; + ColumnPath fallbackValueColumn = ColumnPath.get(fallbackValuePathSegments); + + return new ResolvedVariantPredicate(predicate, fallbackValueColumn); + } + + private static final class PrimitivePredicateFactory { + + private static FilterPredicate toParquetPredicate( + PrimitiveType primitive, String columnPath, Operation op, Object value) { + switch (primitive.getPrimitiveTypeName()) { + case BINARY: + Binary binaryValue = toBinary(primitive, value); + return binaryValue == null ? null : compare(op, FilterApi.binaryColumn(columnPath), binaryValue); + case BOOLEAN: + if (!(value instanceof Boolean)) { + return null; + } + if (op == Operation.EQ) { + return FilterApi.eq(FilterApi.booleanColumn(columnPath), (Boolean) value); + } else if (op == Operation.NOT_EQ) { + return FilterApi.notEq(FilterApi.booleanColumn(columnPath), (Boolean) value); + } + return null; + case INT32: // Handles INT32 and DATE + Integer intValue = asInt32(value); + return intValue == null ? null : compare(op, FilterApi.intColumn(columnPath), intValue); + case INT64: // Handles INT64, TIMESTAMP_MILLIS, TIMESTAMP_MICROS + Long longValue = asInt64(value); + return longValue == null ? null : compare(op, FilterApi.longColumn(columnPath), longValue); + case FLOAT: + Float floatValue = asFloat(op, value); + return floatValue == null ? null : compare(op, FilterApi.floatColumn(columnPath), floatValue); + case DOUBLE: + if (!(value instanceof Number)) { + return null; + } + return compare(op, FilterApi.doubleColumn(columnPath), ((Number) value).doubleValue()); + default: + return null; + } + } + + private static Binary toBinary(PrimitiveType primitive, Object value) { + if (value instanceof Binary binary) { + return binary; + } else if (value instanceof ByteBuffer buffer) { + return Binary.fromReusedByteBuffer(buffer); + } else if (value instanceof byte[] bytes) { + return Binary.fromReusedByteArray(bytes); + } else if (value instanceof CharSequence) { + if (!(primitive.getLogicalTypeAnnotation() instanceof StringLogicalTypeAnnotation)) { + return null; + } + return Binary.fromString(value.toString()); + } + return null; + } + + private static Integer asInt32(Object value) { + Long longValue = toLongExact(value); + if (longValue == null || longValue < Integer.MIN_VALUE || longValue > Integer.MAX_VALUE) { + return null; + } + return longValue.intValue(); + } + + private static Long asInt64(Object value) { + return toLongExact(value); + } + + @SuppressWarnings("checkstyle:CyclomaticComplexity") + private static Long toLongExact(Object value) { + if (!(value instanceof Number)) { + return null; + } + if (value instanceof Double doubleValue) { + if (!Double.isFinite(doubleValue) || doubleValue != Math.rint(doubleValue)) { + return null; + } + if (doubleValue < Long.MIN_VALUE || doubleValue > Long.MAX_VALUE) { + return null; + } + long longValue = doubleValue.longValue(); + return ((double) longValue == doubleValue) ? longValue : null; + } else if (value instanceof Float floatValue) { + if (!Float.isFinite(floatValue) || floatValue != Math.rint(floatValue)) { + return null; + } + double doubleValue = floatValue.doubleValue(); + if (doubleValue < Long.MIN_VALUE || doubleValue > Long.MAX_VALUE) { + return null; + } + long longValue = (long) doubleValue; + return ((double) longValue == doubleValue) ? longValue : null; + } + return ((Number) value).longValue(); + } + + private static Float asFloat(Operation op, Object value) { + if (!(value instanceof Number)) { + return null; + } + double doubleValue = ((Number) value).doubleValue(); + if (!Double.isFinite(doubleValue)) { + return null; + } + float floatValue = (float) doubleValue; + if (op == Operation.EQ || op == Operation.NOT_EQ) { + return ((double) floatValue == doubleValue) ? floatValue : null; + } + return switch (op) { + case GT, GT_EQ -> { + if ((double) floatValue > doubleValue) { + floatValue = Math.nextDown(floatValue); + } + yield floatValue; + } + case LT, LT_EQ -> { + if ((double) floatValue < doubleValue) { + floatValue = Math.nextUp(floatValue); + } + yield floatValue; + } + default -> null; + }; + } + + @SuppressWarnings("checkstyle:MethodTypeParameterName") + private static , COL extends Operators.Column & Operators.SupportsLtGt> + FilterPredicate compare(Operation op, COL column, C value) { + return switch (op) { + case EQ -> FilterApi.eq(column, value); + case NOT_EQ -> FilterApi.notEq(column, value); + case LT -> FilterApi.lt(column, value); + case LT_EQ -> FilterApi.ltEq(column, value); + case GT -> FilterApi.gt(column, value); + case GT_EQ -> FilterApi.gtEq(column, value); + default -> null; + }; + } + } +} diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/parquet/VariantUtil.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/parquet/VariantUtil.java index d03c4a675fa2..736a39b895c8 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/parquet/VariantUtil.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/parquet/VariantUtil.java @@ -21,7 +21,6 @@ import java.util.List; import java.util.Map; -import java.util.function.UnaryOperator; import org.apache.iceberg.Accessor; import org.apache.iceberg.Schema; import org.apache.iceberg.StructLike; @@ -60,8 +59,8 @@ public record VariantField(int fieldId, Accessor accessor, String[] /** * Check if variant shredding is enabled via table properties. */ - public static boolean isVariantShreddingEnabled(UnaryOperator propertyLookup) { - String shreddingEnabled = propertyLookup.apply(InputFormatConfig.VARIANT_SHREDDING_ENABLED); + public static boolean isVariantShreddingEnabled(Map properties) { + String shreddingEnabled = properties.get(InputFormatConfig.VARIANT_SHREDDING_ENABLED); return Boolean.parseBoolean(shreddingEnabled); } @@ -74,7 +73,7 @@ public static boolean isShreddable(Object value) { public static List variantFieldsForShredding( Map properties, Schema schema) { - if (!isVariantShreddingEnabled(properties::get)) { + if (!isVariantShreddingEnabled(properties)) { return List.of(); } return variantFieldsForShredding(schema); @@ -90,8 +89,8 @@ private static List variantFieldsForShredding(Schema schema) { return results; } - public static boolean shouldUseVariantShredding(UnaryOperator propertyLookup, Schema schema) { - return isVariantShreddingEnabled(propertyLookup) && hasVariantFields(schema); + public static boolean shouldUseVariantShredding(Map properties, Schema schema) { + return isVariantShreddingEnabled(properties) && hasVariantFields(schema); } private static boolean hasVariantFields(Schema schema) { diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java index 3044f0467af3..f2abe4cbab35 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java @@ -34,11 +34,13 @@ import org.apache.iceberg.expressions.Not; import org.apache.iceberg.expressions.Or; import org.apache.iceberg.expressions.UnboundPredicate; +import org.apache.iceberg.expressions.UnboundTerm; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.DateTimeUtil; import org.junit.Test; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; public class TestHiveIcebergFilterFactory { @@ -193,6 +195,22 @@ public void testStringType() { assertPredicatesMatch(expected, actual); } + @Test + public void testShreddedVariantPredicate() { + SearchArgument.Builder builder = SearchArgumentFactory.newBuilder(); + SearchArgument arg = builder + .startAnd() + .equals("payload.typed_value.age", PredicateLeaf.Type.LONG, 30L) + .end() + .build(); + + UnboundPredicate expected = + Expressions.equal(Expressions.extract("payload", "$.age", "long"), 30L); + UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg); + + assertPredicatesMatch(expected, actual); + } + @Test public void testFloatType() { SearchArgument.Builder builder = SearchArgumentFactory.newBuilder(); @@ -254,9 +272,20 @@ public void testDecimalType() { assertPredicatesMatch(expected, actual); } - private void assertPredicatesMatch(UnboundPredicate expected, UnboundPredicate actual) { + private void assertPredicatesMatch(UnboundPredicate expected, UnboundPredicate actual) { assertEquals(expected.op(), actual.op()); assertEquals(expected.literal(), actual.literal()); assertEquals(expected.ref().name(), actual.ref().name()); + + // For extract(...) terms, also verify the extracted path matches. + UnboundTerm expectedTerm = expected.term(); + UnboundTerm actualTerm = actual.term(); + if (expectedTerm instanceof org.apache.iceberg.expressions.UnboundExtract expectedExtract) { + assertTrue(actualTerm instanceof org.apache.iceberg.expressions.UnboundExtract); + org.apache.iceberg.expressions.UnboundExtract actualExtract = + (org.apache.iceberg.expressions.UnboundExtract) actualTerm; + assertEquals(expectedExtract.path(), actualExtract.path()); + assertEquals(expectedExtract.type(), actualExtract.type()); + } } } diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergVariant.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergVariant.java deleted file mode 100644 index 71efa4d60899..000000000000 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergVariant.java +++ /dev/null @@ -1,201 +0,0 @@ -/* - * 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.iceberg.mr.hive; - -import java.io.IOException; -import java.util.Collection; -import java.util.List; -import java.util.stream.StreamSupport; -import org.apache.hadoop.fs.Path; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.types.Types; -import org.apache.parquet.hadoop.ParquetFileReader; -import org.apache.parquet.hadoop.util.HadoopInputFile; -import org.apache.parquet.schema.GroupType; -import org.apache.parquet.schema.MessageType; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runners.Parameterized.Parameters; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.Assume.assumeTrue; - -public class TestHiveIcebergVariant extends HiveIcebergStorageHandlerWithEngineBase { - private static final String TYPED_VALUE_FIELD = "typed_value"; - - @Parameters(name = "fileFormat={0}, catalog={1}, isVectorized={2}, formatVersion={3}") - public static Collection parameters() { - return HiveIcebergStorageHandlerWithEngineBase.getParameters(p -> - p.formatVersion() == 3); - } - - @Test - public void testVariantSelectProjection() throws IOException { - assumeParquetNonVectorized(); - - TableIdentifier table = TableIdentifier.of("default", "variant_projection"); - shell.executeStatement(String.format("DROP TABLE IF EXISTS %s", table)); - - shell.executeStatement( - String.format( - "CREATE TABLE %s (id INT, payload VARIANT) STORED BY ICEBERG STORED AS %s %s %s", - table, - fileFormat, - testTables.locationForCreateTableSQL(table), - testTables.propertiesForCreateTableSQL( - ImmutableMap.of("format-version", "3", "variant.shredding.enabled", "true")))); - - shell.executeStatement( - String.format( - "INSERT INTO %s VALUES " + - "(1, parse_json('null'))," + - "(2, parse_json('{\"name\":\"Alice\",\"age\":30}'))," + - "(3, parse_json('{\"name\":\"Bob\"}'))", - table)); - - List rows = - shell.executeStatement( - String.format( - "SELECT id, " + - "variant_get(payload, '$.name') AS name, " + - "try_variant_get(payload, '$.age', 'int') AS age " + - "FROM %s ORDER BY id", - table)); - - Assert.assertEquals(3, rows.size()); - - Assert.assertEquals(1, ((Number) rows.get(0)[0]).intValue()); - Assert.assertNull(rows.get(0)[1]); - Assert.assertNull(rows.get(0)[2]); - - Assert.assertEquals(2, ((Number) rows.get(1)[0]).intValue()); - Assert.assertEquals("Alice", rows.get(1)[1]); - Assert.assertEquals(30, ((Number) rows.get(1)[2]).intValue()); - - Assert.assertEquals(3, ((Number) rows.get(2)[0]).intValue()); - Assert.assertEquals("Bob", rows.get(2)[1]); - Assert.assertNull(rows.get(2)[2]); - - Table icebergTable = testTables.loadTable(table); - Types.NestedField variantField = requiredField(icebergTable, "payload", "Variant column should exist"); - MessageType parquetSchema = readParquetSchema(firstDataFile(icebergTable)); - assertThat(hasTypedValue(parquetSchema, variantField.name())).isTrue(); - } - - @Test - public void testVariantShreddingInStruct() throws IOException { - assumeParquetNonVectorized(); - - TableIdentifier table = TableIdentifier.of("default", "variant_struct_shredding"); - shell.executeStatement(String.format("DROP TABLE IF EXISTS %s", table)); - - shell.executeStatement( - String.format( - "CREATE TABLE %s (id INT, payload STRUCT) STORED BY ICEBERG STORED AS %s %s %s", - table, - fileFormat, - testTables.locationForCreateTableSQL(table), - testTables.propertiesForCreateTableSQL( - ImmutableMap.of("format-version", "3", "variant.shredding.enabled", "true")))); - - shell.executeStatement( - String.format( - "INSERT INTO %s VALUES " + - "(1, named_struct('info', parse_json('null')))," + - "(2, named_struct('info', parse_json('{\"city\":\"Seattle\",\"state\":\"WA\"}')))", - table)); - - Table icebergTable = testTables.loadTable(table); - Types.NestedField payloadField = requiredField(icebergTable, "payload", "Struct column should exist"); - MessageType parquetSchema = readParquetSchema(firstDataFile(icebergTable)); - assertThat(hasTypedValue(parquetSchema, payloadField.name(), "info")).isTrue(); - } - - @Test - public void testVariantShreddingNotAppliedInArrayOrMap() throws IOException { - assumeParquetNonVectorized(); - - TableIdentifier table = TableIdentifier.of("default", "variant_container_no_shredding"); - shell.executeStatement(String.format("DROP TABLE IF EXISTS %s", table)); - - shell.executeStatement( - String.format( - "CREATE TABLE %s (id INT, arr ARRAY, mp MAP) " + - "STORED BY ICEBERG STORED AS %s %s %s", - table, - fileFormat, - testTables.locationForCreateTableSQL(table), - testTables.propertiesForCreateTableSQL( - ImmutableMap.of("format-version", "3", "variant.shredding.enabled", "true")))); - - shell.executeStatement( - String.format( - "INSERT INTO %s VALUES " + - "(1, array(parse_json('{\"a\":1}')), map('k', parse_json('{\"b\":2}')))", - table)); - - Table icebergTable = testTables.loadTable(table); - MessageType parquetSchema = readParquetSchema(firstDataFile(icebergTable)); - // The element/value types should remain as the base VARIANT struct (no typed_value). - assertThat(hasTypedValue(parquetSchema, "arr", "list", "element")).isFalse(); - assertThat(hasTypedValue(parquetSchema, "mp", "key_value", "value")).isFalse(); - } - - private void assumeParquetNonVectorized() { - assumeTrue(fileFormat == FileFormat.PARQUET); - assumeTrue(!isVectorized); - } - - private static Types.NestedField requiredField(Table table, String fieldName, String message) { - Types.NestedField field = table.schema().findField(fieldName); - Assert.assertNotNull(message, field); - return field; - } - - private static DataFile firstDataFile(Table table) { - return StreamSupport.stream(table.currentSnapshot().addedDataFiles(table.io()).spliterator(), false) - .findFirst() - .orElseThrow(() -> new IllegalStateException("No data files written for test table")); - } - - private MessageType readParquetSchema(DataFile dataFile) throws IOException { - Path parquetPath = new Path(dataFile.location()); - try (ParquetFileReader reader = - ParquetFileReader.open(HadoopInputFile.fromPath(parquetPath, shell.getHiveConf()))) { - return reader.getFooter().getFileMetaData().getSchema(); - } - } - - private static GroupType groupAt(MessageType parquetSchema, String... path) { - org.apache.parquet.schema.Type type = parquetSchema.getType(path[0]); - for (int i = 1; i < path.length; i++) { - type = type.asGroupType().getType(path[i]); - } - return type.asGroupType(); - } - - private static boolean hasTypedValue(MessageType parquetSchema, String... pathToVariantGroup) { - return groupAt(parquetSchema, pathToVariantGroup).containsField(TYPED_VALUE_FIELD); - } -} diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergVariantType.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergVariantType.java new file mode 100644 index 000000000000..8f5038325b47 --- /dev/null +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergVariantType.java @@ -0,0 +1,428 @@ +/* + * 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.iceberg.mr.hive; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.stream.StreamSupport; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.mr.hive.variant.VariantPathUtil; +import org.apache.iceberg.parquet.VariantParquetFilters; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.hadoop.util.HadoopInputFile; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.MessageType; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.Test; +import org.junit.runners.Parameterized.Parameters; + +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; + +public class TestHiveIcebergVariantType extends HiveIcebergStorageHandlerWithEngineBase { + + @Parameters(name = "fileFormat={0}, catalog={1}, isVectorized={2}, formatVersion={3}") + public static Collection parameters() { + return HiveIcebergStorageHandlerWithEngineBase.getParameters(p -> + p.formatVersion() == 3); + } + + @Test + public void testVariantShredAndProject() throws IOException { + Schema schema = new Schema( + required(1, "id", Types.IntegerType.get()), + required(2, "payload", Types.VariantType.get())); + + TableIdentifier table = TableIdentifier.of("default", "variant_projection"); + + testTables.createTable(shell, table.name(), + schema, PartitionSpec.unpartitioned(), fileFormat, ImmutableList.of(), formatVersion, + ImmutableMap.of("variant.shredding.enabled", "true")); + + shell.executeStatement( + String.format( + "INSERT INTO %s VALUES " + + "(1, parse_json('null'))," + + "(2, parse_json('{\"name\":\"Alice\",\"age\":30}'))," + + "(3, parse_json('{\"name\":\"Bob\"}'))", + table)); + + String queryStr = "SELECT id, " + + "variant_get(payload, '$.name') AS name, " + + "try_variant_get(payload, '$.age', 'int') AS age " + + "FROM %s ORDER BY id"; + + if (isVectorized) { + List explain = + shell.executeStatement(String.format("EXPLAIN VECTORIZATION " + queryStr, table)); + Assert.assertTrue( + "Expected map-side vectorization for variant_get(payload, ...) query", + mapVectorized(explain)); + } + + List rows = + shell.executeStatement(String.format(queryStr, table)); + + Assert.assertEquals(3, rows.size()); + Assert.assertEquals(1, ((Number) rows.get(0)[0]).intValue()); + Assert.assertNull(rows.get(0)[1]); + Assert.assertNull(rows.get(0)[2]); + + Assert.assertEquals(2, ((Number) rows.get(1)[0]).intValue()); + Assert.assertEquals("Alice", rows.get(1)[1]); + Assert.assertEquals(30, ((Number) rows.get(1)[2]).intValue()); + + Assert.assertEquals(3, ((Number) rows.get(2)[0]).intValue()); + Assert.assertEquals("Bob", rows.get(2)[1]); + Assert.assertNull(rows.get(2)[2]); + + Table icebergTable = testTables.loadTable(table); + Types.NestedField variantField = requiredField(icebergTable, "payload", "Variant column should exist"); + MessageType parquetSchema = readParquetSchema(firstDataFile(icebergTable)); + assertThat(hasTypedValue(parquetSchema, variantField.name())).isTrue(); + } + + @Test + public void testVariantShreddingAppliedToStructField() throws IOException { + Schema schema = new Schema( + required(1, "id", Types.IntegerType.get()), + required(2, "payload", Types.StructType.of( + required(3, "info", Types.VariantType.get())))); + + TableIdentifier table = TableIdentifier.of("default", "variant_struct_shredding"); + + testTables.createTable(shell, table.name(), + schema, PartitionSpec.unpartitioned(), fileFormat, ImmutableList.of(), formatVersion, + ImmutableMap.of("variant.shredding.enabled", "true")); + + shell.executeStatement( + String.format( + "INSERT INTO %s VALUES " + + "(1, named_struct('info', parse_json('null')))," + + "(2, named_struct('info', parse_json('{\"city\":\"Seattle\",\"state\":\"WA\"}')))", + table)); + + String queryStr = "SELECT id, " + + "variant_get(payload.info, '$.city') AS city, " + + "variant_get(payload.info, '$.state') AS state " + + "FROM %s ORDER BY id"; + + if (isVectorized) { + List explain = + shell.executeStatement(String.format("EXPLAIN VECTORIZATION " + queryStr, table)); + Assert.assertTrue( + "Expected map-side vectorization for nested variant_get(payload.info, ...) query", + mapVectorized(explain)); + } + + List rows = + shell.executeStatement(String.format(queryStr, table)); + + Assert.assertEquals(2, rows.size()); + Assert.assertEquals(1, ((Number) rows.get(0)[0]).intValue()); + Assert.assertNull(rows.get(0)[1]); + Assert.assertNull(rows.get(0)[2]); + Assert.assertEquals(2, ((Number) rows.get(1)[0]).intValue()); + Assert.assertEquals("Seattle", rows.get(1)[1]); + Assert.assertEquals("WA", rows.get(1)[2]); + + Table icebergTable = testTables.loadTable(table); + Types.NestedField payloadField = requiredField(icebergTable, "payload", "Struct column should exist"); + MessageType parquetSchema = readParquetSchema(firstDataFile(icebergTable)); + assertThat(hasTypedValue(parquetSchema, payloadField.name(), "info")).isTrue(); + } + + @Test + public void testVariantInContainersIsNotShredded() throws IOException { + Assume.assumeTrue(isVectorized); + + Schema schema = new Schema( + required(1, "id", Types.IntegerType.get()), + required(2, "arr", Types.ListType.ofRequired(3, Types.VariantType.get())), + required(4, "mp", Types.MapType.ofRequired(5, 6, + Types.StringType.get(), Types.VariantType.get()))); + + TableIdentifier table = TableIdentifier.of("default", "variant_container_no_shredding"); + + testTables.createTable(shell, table.name(), + schema, PartitionSpec.unpartitioned(), fileFormat, ImmutableList.of(), formatVersion, + ImmutableMap.of("variant.shredding.enabled", "true")); + + shell.executeStatement( + String.format( + "INSERT INTO %s VALUES " + + "(1, array(parse_json('{\"a\":1}')), map('k', parse_json('{\"b\":2}')))", + table)); + + Table icebergTable = testTables.loadTable(table); + MessageType parquetSchema = readParquetSchema(firstDataFile(icebergTable)); + // The element/value types should remain as the base VARIANT struct (no typed_value). + assertThat(hasTypedValue(parquetSchema, "arr", "list", "element")).isFalse(); + assertThat(hasTypedValue(parquetSchema, "mp", "key_value", "value")).isFalse(); + } + + @Test + public void testParquetRowGroupPruningWithVariantPredicate() { + Schema schema = new Schema( + required(1, "id", Types.IntegerType.get()), + required(2, "payload", Types.VariantType.get())); + + TableIdentifier table = TableIdentifier.of("default", "variant_select"); + + testTables.createTable(shell, table.name(), + schema, PartitionSpec.unpartitioned(), fileFormat, ImmutableList.of(), formatVersion, + ImmutableMap.of( + "variant.shredding.enabled", "true", + // Force multiple row groups so we can assert row-group pruning for variant predicates. + "write.parquet.row-group-size-bytes", "65536")); + + // Insert enough data (with padding) to create multiple row groups, and group tiers so row groups can be pruned. + int rowsPerTier = 100; + int padLen = 2000; + StringBuilder insert = new StringBuilder("INSERT INTO ") + .append(table) + .append(" VALUES "); + for (int i = 1; i <= rowsPerTier * 2; i++) { + String tier = i <= rowsPerTier ? "gold" : "silver"; + if (i > 1) { + insert.append(","); + } + insert.append("(") + .append(i) + .append(", parse_json(concat('{\"tier\":\"") + .append(tier) + .append("\",\"pad\":\"', repeat('x', ") + .append(padLen) + .append("), '\"}')))"); + } + shell.executeStatement(insert.toString()); + + String queryStr = "SELECT id FROM %s " + + "WHERE variant_get(payload, '$.tier') = 'gold' " + + "ORDER BY id"; + + if (isVectorized) { + List explain = + shell.executeStatement(String.format("EXPLAIN VECTORIZATION " + queryStr, table)); + Assert.assertTrue( + "Expected map-side vectorization for variant predicate query", + mapVectorized(explain)); + } + + List rows = + shell.executeStatement(String.format(queryStr, table)); + + Assert.assertEquals(rowsPerTier, rows.size()); + Assert.assertEquals(1, ((Number) rows.get(0)[0]).intValue()); + Assert.assertEquals(rowsPerTier, ((Number) rows.get(rows.size() - 1)[0]).intValue()); + + // Assert the file has multiple row groups and that our Parquet reader row-group pruning will skip some. + Table icebergTable = testTables.loadTable(table); + DataFile dataFile = firstDataFile(icebergTable); + Path parquetPath = new Path(dataFile.location()); + int rowGroupCount; + try (ParquetFileReader reader = + ParquetFileReader.open(HadoopInputFile.fromPath(parquetPath, shell.getHiveConf()))) { + rowGroupCount = reader.getRowGroups().size(); + } catch (IOException e) { + throw new RuntimeException(e); + } + Assert.assertTrue( + "Expected multiple row groups to validate pruning", + rowGroupCount > 1); + + Expression filter = Expressions.equal( + Expressions.extract("payload", "$.tier", "string"), + "gold"); + if (isVectorized) { + assertVectorizedParquetRowGroupsPruned(parquetPath, filter); + } else { + assertNonVectorizedParquetRowGroupsPruned(parquetPath, filter); + } + } + + @Test + public void testVariantProjectionWithColumnRename() { + Assume.assumeTrue(isVectorized); + + Schema schema = new Schema( + required(1, "id", Types.IntegerType.get()), + required(2, "payload", Types.VariantType.get())); + + TableIdentifier table = TableIdentifier.of("default", "variant_rename"); + + testTables.createTable(shell, table.name(), + schema, PartitionSpec.unpartitioned(), fileFormat, ImmutableList.of(), formatVersion, + ImmutableMap.of("variant.shredding.enabled", "true")); + + shell.executeStatement( + String.format( + "INSERT INTO %s VALUES (1, parse_json('{\"a\": 100}'))", + table)); + + shell.executeStatement( + String.format( + "ALTER TABLE %s CHANGE COLUMN payload data VARIANT", + table)); + + List rows = shell.executeStatement( + String.format( + "SELECT id, try_variant_get(data, '$.a', 'int') FROM %s", + table)); + + Assert.assertEquals(1, rows.size()); + Assert.assertEquals(1, ((Number) rows.get(0)[0]).intValue()); + Assert.assertEquals(100, ((Number) rows.get(0)[1]).intValue()); + } + + private static boolean mapVectorized(List explain) { + boolean insideMapSection = false; + for (Object[] row : explain) { + if (row == null || row.length == 0 || row[0] == null) { + continue; + } + + String line = row[0].toString().trim(); + // Enter Map Vectorization section + if (line.startsWith("Map Vectorization:")) { + insideMapSection = true; + continue; + } + if (insideMapSection) { + // Check for vectorized=true inside the section + if (line.contains("vectorized: true")) { + return true; + } + // Exit section when a Reduce vertex or Reduce Vectorization starts + if (line.startsWith("Reducer") || line.startsWith("Reduce Vectorization:")) { + insideMapSection = false; + } + } + } + return false; + } + + private static void assertVectorizedParquetRowGroupsPruned(Path parquetPath, Expression filter) { + assertParquetRowGroupsPruned( + parquetPath, filter, + (parquetMetadata, fileSchema, expr) -> + // Simulate what HiveVectorizedReader.parquetRecordReader() does + VariantParquetFilters + .pruneVariantRowGroups(parquetMetadata, fileSchema, expr) + .getBlocks() + .size()); + } + + private static void assertNonVectorizedParquetRowGroupsPruned(Path parquetPath, Expression filter) { + assertParquetRowGroupsPruned( + parquetPath, filter, + (parquetMetadata, fileSchema, expr) -> { + // Simulate what ReadConf does - uses variantRowGroupMayMatch to compute shouldSkip array + boolean[] mayMatch = VariantParquetFilters + .variantRowGroupMayMatch(fileSchema, expr, parquetMetadata.getBlocks()); + int matching = 0; + for (boolean match : mayMatch) { + if (match) { + matching++; + } + } + return matching; + }); + } + + @FunctionalInterface + private interface RowGroupPruner { + int prune( + ParquetMetadata parquetMetadata, + MessageType fileSchema, + Expression filter) throws Exception; + } + + private static void assertParquetRowGroupsPruned( + Path parquetPath, + Expression filter, + RowGroupPruner rowGroupPruner) { + try (ParquetFileReader reader = + ParquetFileReader.open(HadoopInputFile.fromPath(parquetPath, shell.getHiveConf()))) { + ParquetMetadata parquetMetadata = reader.getFooter(); + MessageType fileSchema = parquetMetadata.getFileMetaData().getSchema(); + int originalRowGroups = parquetMetadata.getBlocks().size(); + + Assert.assertTrue( + "Expected multiple row groups to validate pruning", + originalRowGroups > 1); + + int matchingRowGroups = rowGroupPruner.prune(parquetMetadata, fileSchema, filter); + + Assert.assertTrue( + "Expected at least one row group to be pruned", + matchingRowGroups > 0 && matchingRowGroups < originalRowGroups); + + } catch (Exception e) { + throw new RuntimeException("Unable to validate vectorized Parquet row-group pruning", e); + } + } + + private static Types.NestedField requiredField(Table table, String fieldName, String message) { + Types.NestedField field = table.schema().findField(fieldName); + Assert.assertNotNull(message, field); + return field; + } + + private static DataFile firstDataFile(Table table) { + return StreamSupport.stream(table.currentSnapshot().addedDataFiles(table.io()).spliterator(), false) + .findFirst() + .orElseThrow(() -> new IllegalStateException("No data files written for test table")); + } + + private MessageType readParquetSchema(DataFile dataFile) throws IOException { + Path parquetPath = new Path(dataFile.location()); + try (ParquetFileReader reader = + ParquetFileReader.open(HadoopInputFile.fromPath(parquetPath, shell.getHiveConf()))) { + return reader.getFooter().getFileMetaData().getSchema(); + } + } + + private static GroupType groupAt(MessageType parquetSchema, String... path) { + org.apache.parquet.schema.Type type = parquetSchema.getType(path[0]); + for (int i = 1; i < path.length; i++) { + type = type.asGroupType().getType(path[i]); + } + return type.asGroupType(); + } + + private static boolean hasTypedValue(MessageType parquetSchema, String... pathToVariantGroup) { + return groupAt(parquetSchema, pathToVariantGroup).containsField( + VariantPathUtil.TYPED_VALUE); + } +} diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/variant/TestVariantFilterRewriter.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/variant/TestVariantFilterRewriter.java new file mode 100644 index 000000000000..61b6c67de83a --- /dev/null +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/variant/TestVariantFilterRewriter.java @@ -0,0 +1,184 @@ +/* + * 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.iceberg.mr.hive.variant; + +import java.util.List; +import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFVariantGet; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.UnboundPredicate; +import org.junit.Assert; +import org.junit.Test; + +public class TestVariantFilterRewriter { + + private static final TypeInfo VARIANT_STRUCT = + TypeInfoUtils.getTypeInfoFromTypeString("struct"); + + @Test + public void testStripExtractPredicateBecomesAlwaysTrue() { + Expression expr = Expressions.equal( + Expressions.extract("payload", "$.tier", "string"), + "gold"); + + Assert.assertSame( + Expressions.alwaysTrue(), + VariantFilterRewriter.stripVariantExtractPredicates(expr)); + } + + @Test + public void testStripTypedValueReferenceBecomesAlwaysTrue() { + Expression expr = Expressions.equal("payload.typed_value.tier", "gold"); + + Assert.assertSame( + Expressions.alwaysTrue(), + VariantFilterRewriter.stripVariantExtractPredicates(expr)); + } + + @Test + public void testStripExtractFromAndKeepsOtherConjunct() { + Expression expr = Expressions.and( + Expressions.equal( + Expressions.extract("payload", "$.tier", "string"), + "gold"), + Expressions.greaterThanOrEqual("id", 5)); + + Expression stripped = VariantFilterRewriter.stripVariantExtractPredicates(expr); + Assert.assertTrue(stripped instanceof UnboundPredicate); + + UnboundPredicate predicate = (UnboundPredicate) stripped; + Assert.assertEquals(Expression.Operation.GT_EQ, predicate.op()); + Assert.assertEquals("id", predicate.ref().name()); + } + + @Test + public void testStripExtractUnderNotBecomesAlwaysTrue() { + Expression expr = Expressions.not( + Expressions.equal( + Expressions.extract("payload", "$.tier", "string"), + "gold")); + + Assert.assertSame( + Expressions.alwaysTrue(), + VariantFilterRewriter.stripVariantExtractPredicates(expr)); + } + + @Test + public void testStripExtractFromOrKeepsOtherDisjunct() { + Expression expr = Expressions.or( + Expressions.equal( + Expressions.extract("payload", "$.tier", "string"), + "gold"), + Expressions.lessThan("id", 10)); + + Expression stripped = VariantFilterRewriter.stripVariantExtractPredicates(expr); + Assert.assertTrue(stripped instanceof UnboundPredicate); + + UnboundPredicate predicate = (UnboundPredicate) stripped; + Assert.assertEquals(Expression.Operation.LT, predicate.op()); + Assert.assertEquals("id", predicate.ref().name()); + } + + @Test + public void testRewritesVariantGetToShreddedColumn() throws Exception { + ExprNodeGenericFuncDesc predicate = equals( + variantGet("payload", "$.tier"), + stringConst("gold")); + + ExprNodeGenericFuncDesc rewritten = VariantFilterRewriter.rewriteForShredding(predicate); + Assert.assertNotNull(rewritten); + + // original must remain unchanged (rewriteForShredding clones) + Assert.assertTrue(predicate.getChildren().getFirst() instanceof ExprNodeGenericFuncDesc); + + ExprNodeDesc lhs = rewritten.getChildren().getFirst(); + Assert.assertTrue(lhs instanceof ExprNodeColumnDesc); + + ExprNodeColumnDesc col = (ExprNodeColumnDesc) lhs; + Assert.assertEquals("payload.typed_value.tier", col.getColumn()); + Assert.assertEquals("t", col.getTabAlias()); + Assert.assertEquals(predicate.getChildren().getFirst().getTypeInfo(), col.getTypeInfo()); + } + + @Test + public void testRewritesNestedObjectPath() throws Exception { + ExprNodeGenericFuncDesc predicate = equals( + variantGet("payload", "$.a.b"), + stringConst("gold")); + + ExprNodeGenericFuncDesc rewritten = VariantFilterRewriter.rewriteForShredding(predicate); + + ExprNodeDesc lhs = rewritten.getChildren().getFirst(); + Assert.assertTrue(lhs instanceof ExprNodeColumnDesc); + Assert.assertEquals("payload.typed_value.a.b", ((ExprNodeColumnDesc) lhs).getColumn()); + } + + @Test + public void testDoesNotRewriteArrayPath() throws Exception { + ExprNodeGenericFuncDesc predicate = equals( + variantGet("payload", "$[0]"), + stringConst("gold")); + + ExprNodeGenericFuncDesc rewritten = VariantFilterRewriter.rewriteForShredding(predicate); + + ExprNodeDesc lhs = rewritten.getChildren().getFirst(); + Assert.assertTrue(lhs instanceof ExprNodeGenericFuncDesc); + Assert.assertTrue(((ExprNodeGenericFuncDesc) lhs).getGenericUDF() instanceof GenericUDFVariantGet); + } + + @Test + public void testDoesNotRewriteWhenPathIsNotConstant() throws Exception { + ExprNodeColumnDesc payload = new ExprNodeColumnDesc( + VARIANT_STRUCT, "payload", "t", false); + ExprNodeColumnDesc path = new ExprNodeColumnDesc( + TypeInfoFactory.stringTypeInfo, "path", "t", false); + ExprNodeGenericFuncDesc udf = ExprNodeGenericFuncDesc.newInstance( + new GenericUDFVariantGet(), List.of(payload, path)); + ExprNodeGenericFuncDesc predicate = equals(udf, stringConst("gold")); + + ExprNodeGenericFuncDesc rewritten = VariantFilterRewriter.rewriteForShredding(predicate); + ExprNodeDesc lhs = rewritten.getChildren().getFirst(); + Assert.assertTrue(lhs instanceof ExprNodeGenericFuncDesc); + Assert.assertTrue(((ExprNodeGenericFuncDesc) lhs).getGenericUDF() instanceof GenericUDFVariantGet); + } + + private static ExprNodeGenericFuncDesc equals(ExprNodeDesc left, ExprNodeDesc right) throws Exception { + return ExprNodeGenericFuncDesc.newInstance(new GenericUDFOPEqual(), List.of(left, right)); + } + + private static ExprNodeConstantDesc stringConst(String value) { + return new ExprNodeConstantDesc(TypeInfoFactory.stringTypeInfo, value); + } + + private static ExprNodeGenericFuncDesc variantGet(String column, String jsonPath) throws Exception { + ExprNodeColumnDesc payload = new ExprNodeColumnDesc(VARIANT_STRUCT, column, "t", false); + ExprNodeConstantDesc path = new ExprNodeConstantDesc(TypeInfoFactory.stringTypeInfo, jsonPath); + return ExprNodeGenericFuncDesc.newInstance(new GenericUDFVariantGet(), List.of(payload, path)); + } + +} diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/vector/TestParquetVariantRecordReader.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/vector/TestParquetVariantRecordReader.java new file mode 100644 index 000000000000..57ca122bcd7e --- /dev/null +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/vector/TestParquetVariantRecordReader.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.iceberg.mr.hive.vector; + +import java.util.List; +import java.util.Set; +import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; +import org.apache.hadoop.mapred.JobConf; +import org.apache.iceberg.Schema; +import org.apache.iceberg.mr.hive.variant.VariantProjectionUtil.VariantColumnDescriptor; +import org.apache.iceberg.mr.hive.variant.VariantProjectionUtil.VariantProjection; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.Types; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.MessageTypeParser; +import org.junit.Assert; +import org.junit.Test; + +public class TestParquetVariantRecordReader { + + private static final String SIMPLE_SCHEMA = + "message root {" + + " optional group col {" + + " required binary metadata;" + + " required binary value;" + + " optional group typed_value {" + + " optional binary city (STRING);" + + " optional int32 age;" + + " }" + + " }" + + "}"; + + private static final String STRUCT_SCHEMA = + "message root {" + + " optional group col {" + + " required binary metadata;" + + " required binary value;" + + " optional group typed_value {" + + " optional group address {" + + " optional binary city (STRING);" + + " optional int32 zip;" + + " }" + + " optional binary name (STRING);" + + " }" + + " }" + + "}"; + + @Test + public void testProjectionExcludesUnprojectedColumns() { + VariantProjection projection = + createProjection(SIMPLE_SCHEMA, List.of("col.city")); + + Assert.assertNotNull(projection); + Set leaves = getLeafNames(projection.requestedColumns()); + + Assert.assertTrue("Should include metadata", leaves.contains("metadata")); + Assert.assertTrue("Should include value", leaves.contains("value")); + Assert.assertTrue("Should include projected field city", leaves.contains("city")); + Assert.assertFalse("Should NOT include unprojected field age", leaves.contains("age")); + Assert.assertEquals("Total columns", 3, projection.requestedColumns().size()); + } + + @Test + public void testNoProjectionFetchesAllColumns() { + VariantProjection projection = + createProjection(SIMPLE_SCHEMA, List.of()); + + Assert.assertNotNull(projection); + Assert.assertEquals("Should read all fields when no projection", 4, projection.requestedColumns().size()); + } + + @Test + public void testProjectionCombinesSelectAndFilter() { + // Simulates: SELECT variant_get(col, '$.city') WHERE variant_get(col, '$.age') > 30 + String schemaStr = "message root {" + + " optional group col {" + + " required binary metadata;" + + " required binary value;" + + " optional group typed_value {" + + " optional binary city (STRING);" + + " optional int32 age;" + + " optional binary name (STRING);" + + " }" + + " }" + + "}"; + + VariantProjection projection = + createProjection(schemaStr, List.of("col.city", "col.age")); + + Assert.assertNotNull(projection); + Set leaves = getLeafNames(projection.requestedColumns()); + + Assert.assertTrue("Should include city from SELECT", leaves.contains("city")); + Assert.assertTrue("Should include age from WHERE", leaves.contains("age")); + Assert.assertFalse("Should NOT include unreferenced name", leaves.contains("name")); + Assert.assertEquals("Total columns", 4, projection.requestedColumns().size()); + } + + @Test + public void testStructProjectionIncludesAllChildren() { + VariantProjection projection = + createProjection(STRUCT_SCHEMA, List.of("col.address")); + + Assert.assertNotNull(projection); + Set leaves = getLeafNames(projection.requestedColumns()); + + Assert.assertTrue("Should include all children of address", leaves.contains("city")); + Assert.assertTrue("Should include all children of address", leaves.contains("zip")); + Assert.assertEquals("Total columns", 4, projection.requestedColumns().size()); + } + + @Test + public void testProjectionOfFieldInStruct() { + VariantProjection projection = + createProjection(STRUCT_SCHEMA, List.of("col.address.city")); + + Assert.assertNotNull(projection); + Set leaves = getLeafNames(projection.requestedColumns()); + + Assert.assertTrue("Should include address.city", leaves.contains("city")); + Assert.assertFalse("Should NOT include address.zip", leaves.contains("zip")); + Assert.assertFalse("Should NOT include name", leaves.contains("name")); + Assert.assertEquals("Total columns", 3, projection.requestedColumns().size()); + } + + @Test + public void testProjectionOfVariantFieldInStruct() { + String schemaStr = "message root {" + + " optional group top_struct {" + + " optional group variant_col {" + + " required binary metadata;" + + " required binary value;" + + " optional group typed_value {" + + " optional binary foo (STRING);" + + " optional binary bar (STRING);" + + " }" + + " }" + + " optional int32 other_col;" + + " }" + + "}"; + + MessageType fileSchema = MessageTypeParser.parseMessageType(schemaStr); + + Schema icebergSchema = new Schema( + Types.NestedField.optional(1, "top_struct", Types.StructType.of( + Types.NestedField.optional(2, "variant_col", Types.VariantType.get()), + Types.NestedField.optional(3, "other_col", Types.IntegerType.get()) + )) + ); + + JobConf job = new JobConf(); + job.set(org.apache.hadoop.hive.ql.io.IOConstants.COLUMNS, "top_struct"); + ColumnProjectionUtils.appendReadColumns( + job, + List.of(0), + List.of("top_struct"), + List.of("top_struct.variant_col.foo"), + false); + + VariantProjection projection = + VariantProjection.create(fileSchema, job, icebergSchema); + + Assert.assertNotNull(projection); + Set leaves = getLeafNames(projection.requestedColumns()); + + Assert.assertTrue("Should include variant_col.foo", leaves.contains("foo")); + Assert.assertFalse("Should NOT include variant_col.bar", leaves.contains("bar")); + Assert.assertEquals("Total columns", 3, projection.requestedColumns().size()); + + VariantColumnDescriptor desc = projection.variantColumns().get(0); + Assert.assertArrayEquals("Field path should point to variant_col index", new int[]{0}, desc.fieldPath()); + } + + @Test + public void testProjectionOnMultipleVariantColumns() { + String schemaStr = "message root {" + + " optional group col1 {" + + " required binary metadata;" + + " required binary value;" + + " optional group typed_value {" + + " optional binary a (STRING);" + + " optional binary b (STRING);" + + " }" + + " }" + + " optional group col2 {" + + " required binary metadata;" + + " required binary value;" + + " optional group typed_value {" + + " optional int32 x;" + + " optional int32 y;" + + " }" + + " }" + + "}"; + MessageType fileSchema = MessageTypeParser.parseMessageType(schemaStr); + + Schema icebergSchema = new Schema( + Types.NestedField.optional(1, "col1", Types.VariantType.get()), + Types.NestedField.optional(2, "col2", Types.VariantType.get()) + ); + + JobConf job = new JobConf(); + job.set(org.apache.hadoop.hive.ql.io.IOConstants.COLUMNS, "col1,col2"); + ColumnProjectionUtils.appendReadColumns( + job, + List.of(0, 1), + List.of("col1", "col2"), + List.of("col1.a", "col2.x"), + false); + + VariantProjection projection = + VariantProjection.create(fileSchema, job, icebergSchema); + + Assert.assertNotNull(projection); + + Set col1Leaves = Sets.newHashSet(); + Set col2Leaves = Sets.newHashSet(); + + for (ColumnDescriptor desc : projection.requestedColumns()) { + String[] path = desc.getPath(); + String leaf = path[path.length - 1]; + + if ("col1".equals(path[0])) { + col1Leaves.add(leaf); + } else if ("col2".equals(path[0])) { + col2Leaves.add(leaf); + } + } + + Assert.assertTrue("col1 should include a", col1Leaves.contains("a")); + Assert.assertFalse("col1 should NOT include b", col1Leaves.contains("b")); + Assert.assertTrue("col2 should include x", col2Leaves.contains("x")); + Assert.assertFalse("col2 should NOT include y", col2Leaves.contains("y")); + Assert.assertEquals("Total columns", 6, projection.requestedColumns().size()); + } + + @Test + public void testPrunedSchemaMatchesProjection() { + VariantProjection projection = + createProjection(SIMPLE_SCHEMA, List.of("col.city")); + + Assert.assertNotNull(projection); + Assert.assertEquals(1, projection.variantColumns().size()); + + VariantColumnDescriptor desc = projection.variantColumns().get(0); + org.apache.parquet.schema.Type pruned = desc.prunedSchema(); + + org.apache.parquet.schema.GroupType prunedGroup = pruned.asGroupType(); + Assert.assertTrue("Should contain metadata", prunedGroup.containsField("metadata")); + Assert.assertTrue("Should contain value", prunedGroup.containsField("value")); + + org.apache.parquet.schema.GroupType typedValue = prunedGroup.getType("typed_value").asGroupType(); + Assert.assertTrue("Should contain city", typedValue.containsField("city")); + Assert.assertFalse("Should NOT contain age", typedValue.containsField("age")); + } + + @Test + public void testProjectionSupportsTopLevelRename() { + String schemaStr = "message root {" + + " optional group old_col = 1 {" + // Parquet has old_name + " required binary metadata;" + + " required binary value;" + + " optional group typed_value {" + + " optional binary city (STRING);" + + " optional int32 age;" + + " }" + + " }" + + "}"; + MessageType fileSchema = MessageTypeParser.parseMessageType(schemaStr); + + // Iceberg has new_name with same ID + Schema icebergSchema = new Schema(Types.NestedField.optional(1, "new_col", Types.VariantType.get())); + + JobConf job = new JobConf(); + job.set(org.apache.hadoop.hive.ql.io.IOConstants.COLUMNS, "new_col"); + ColumnProjectionUtils.appendReadColumns( + job, + List.of(0), + List.of("new_col"), + List.of("new_col.city"), // Hive asks for new name + false); + + VariantProjection projection = + VariantProjection.create(fileSchema, job, icebergSchema); + + Assert.assertNotNull(projection); + Assert.assertEquals(1, projection.variantColumns().size()); + + VariantColumnDescriptor desc = projection.variantColumns().get(0); + // Path should be physical (resolved from rename) + Assert.assertEquals("old_col", desc.physicalPath()[0]); + + org.apache.parquet.schema.Type pruned = desc.prunedSchema(); + // Pruned schema should be based on physical type but pruned using logical paths + // The root name of pruned schema comes from physical type ("old_col") + Assert.assertEquals("old_col", pruned.getName()); + + org.apache.parquet.schema.GroupType typedValue = pruned.asGroupType().getType("typed_value").asGroupType(); + Assert.assertTrue("Should contain city", typedValue.containsField("city")); + Assert.assertFalse("Should NOT contain age", typedValue.containsField("age")); + } + + @Test + public void testUnshreddedVariantIgnored() { + String schemaStr = "message root {" + + " optional group col {" + + " required binary metadata;" + + " required binary value;" + + " }" + + "}"; + MessageType fileSchema = MessageTypeParser.parseMessageType(schemaStr); + Schema icebergSchema = new Schema(Types.NestedField.optional(1, "col", Types.VariantType.get())); + + JobConf job = new JobConf(); + job.set(org.apache.hadoop.hive.ql.io.IOConstants.COLUMNS, "col"); + // Even if we request a path, if it's unshredded, we can't project + ColumnProjectionUtils.appendReadColumns( + job, List.of(0), List.of("col"), List.of("col.city"), false); + + VariantProjection projection = + VariantProjection.create(fileSchema, job, icebergSchema); + + // Should return null (no shredded columns found) + Assert.assertNull("Should return null for unshredded variant", projection); + } + + @Test + public void testListProjectionPruning() { + // Tests that projecting a field inside a LIST retrieves the entire list structure + // but still prunes siblings outside the list. + String schemaStr = "message root {" + + " optional group col {" + + " required binary metadata;" + + " required binary value;" + + " optional group typed_value {" + + " optional group my_list (LIST) {" + + " repeated group list {" + + " optional group element {" + + " optional int32 item_id;" + + " optional binary item_name (STRING);" + + " }" + + " }" + + " }" + + " optional int32 other_field;" + + " }" + + " }" + + "}"; + + VariantProjection projection = + createProjection(schemaStr, List.of("col.my_list.item_id")); + + Assert.assertNotNull(projection); + Set leaves = getLeafNames(projection.requestedColumns()); + + // Standard fields + Assert.assertTrue("Should include metadata", leaves.contains("metadata")); + Assert.assertTrue("Should include value", leaves.contains("value")); + + // List fields - ALL should be present because we don't prune inside LIST + Assert.assertTrue("Should include requested item_id", leaves.contains("item_id")); + Assert.assertTrue("Should include unrequested item_name (sibling in list)", leaves.contains("item_name")); + + // Outer sibling - SHOULD be pruned + Assert.assertFalse("Should NOT include other_field", leaves.contains("other_field")); + } + + @Test + public void testMapProjectionPruning() { + // Tests that projecting a field inside a MAP retrieves the entire map structure + String schemaStr = "message root {" + + " optional group col {" + + " required binary metadata;" + + " required binary value;" + + " optional group typed_value {" + + " optional group my_map (MAP) {" + + " repeated group key_value {" + + " required binary key (STRING);" + + " optional group value {" + + " optional int32 map_val;" + + " optional int32 map_extra;" + + " }" + + " }" + + " }" + + " optional int32 other_field;" + + " }" + + " }" + + "}"; + + VariantProjection projection = + createProjection(schemaStr, List.of("col.my_map.map_val")); + + Assert.assertNotNull(projection); + Set leaves = getLeafNames(projection.requestedColumns()); + + // Map fields - ALL should be present + Assert.assertTrue("Should include map key", leaves.contains("key")); + Assert.assertTrue("Should include requested map_val", leaves.contains("map_val")); + Assert.assertTrue("Should include unrequested map_extra", leaves.contains("map_extra")); + + // Outer sibling - SHOULD be pruned + Assert.assertFalse("Should NOT include other_field", leaves.contains("other_field")); + } + + private static VariantProjection createProjection( + String schemaStr, List nestedPaths) { + MessageType fileSchema = MessageTypeParser.parseMessageType(schemaStr); + Schema icebergSchema = new Schema(Types.NestedField.optional(1, "col", Types.VariantType.get())); + + JobConf job = new JobConf(); + job.set(org.apache.hadoop.hive.ql.io.IOConstants.COLUMNS, "col"); + ColumnProjectionUtils.appendReadColumns(job, List.of(0), List.of("col"), nestedPaths, false); + + return VariantProjection.create(fileSchema, job, icebergSchema); + } + + private static Set getLeafNames(List requested) { + Set leaves = Sets.newHashSet(); + for (ColumnDescriptor desc : requested) { + leaves.add(desc.getPath()[desc.getPath().length - 1]); + } + return leaves; + } +} diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/parquet/TestVariantParquetFilters.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/parquet/TestVariantParquetFilters.java new file mode 100644 index 000000000000..de2ee04764aa --- /dev/null +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/parquet/TestVariantParquetFilters.java @@ -0,0 +1,376 @@ +/* + * 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.iceberg.parquet; + +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.parquet.filter2.compat.FilterCompat; +import org.apache.parquet.filter2.predicate.FilterApi; +import org.apache.parquet.filter2.predicate.FilterPredicate; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.MessageTypeParser; +import org.junit.Assert; +import org.junit.Test; + +public class TestVariantParquetFilters { + + private static final MessageType VARIANT_MESSAGE = + MessageTypeParser.parseMessageType( + "message root {" + + " optional group payload {" + + " required binary metadata;" + + " optional binary value;" + + " optional group typed_value {" + + " required group tier {" + + " optional binary value;" + + " optional binary typed_value (UTF8);" + + " }" + + " }" + + " }" + + "}"); + + private static final MessageType VARIANT_MESSAGE_WITH_LONG = + MessageTypeParser.parseMessageType( + "message root {" + + " optional group payload {" + + " required binary metadata;" + + " optional binary value;" + + " optional group typed_value {" + + " required group age {" + + " optional binary value;" + + " optional int64 typed_value;" + + " }" + + " }" + + " }" + + "}"); + + private static final MessageType VARIANT_MESSAGE_NESTED = + MessageTypeParser.parseMessageType( + "message root {" + + " optional group payload {" + + " required binary metadata;" + + " optional binary value;" + + " optional group typed_value {" + + " required group a {" + + " required group b {" + + " optional binary value;" + + " optional binary typed_value (UTF8);" + + " }" + + " }" + + " }" + + " }" + + "}"); + + private static final MessageType VARIANT_MESSAGE_STRUCT_NESTED = + MessageTypeParser.parseMessageType( + "message root {" + + " optional group payload {" + + " optional group info {" + + " required binary metadata;" + + " optional binary value;" + + " optional group typed_value {" + + " required group city {" + + " optional binary value;" + + " optional binary typed_value (UTF8);" + + " }" + + " }" + + " }" + + " }" + + "}"); + + private static final MessageType VARIANT_MESSAGE_MULTI_TYPE = + MessageTypeParser.parseMessageType( + "message root {" + + " optional group payload {" + + " required binary metadata;" + + " optional binary value;" + + " optional group typed_value {" + + " required group int_field {" + + " optional binary value;" + + " optional int32 typed_value;" + + " }" + + " required group float_field {" + + " optional binary value;" + + " optional float typed_value;" + + " }" + + " required group double_field {" + + " optional binary value;" + + " optional double typed_value;" + + " }" + + " required group bool_field {" + + " optional binary value;" + + " optional boolean typed_value;" + + " }" + + " }" + + " }" + + "}"); + + @Test + public void testEqualsPredicateConversion() { + assertPredicateConversion( + Expressions.equal("payload.typed_value.tier", "gold"), + VARIANT_MESSAGE, + FilterApi.eq( + FilterApi.binaryColumn("payload.typed_value.tier.typed_value"), + Binary.fromString("gold"))); + } + + @Test + public void testUnknownColumn() { + Expression expr = Expressions.equal("payload.typed_value.unknown", "gold"); + FilterCompat.Filter filter = VariantParquetFilters.toParquetFilter(VARIANT_MESSAGE, expr); + Assert.assertNull(filter); + } + + @Test + public void testLongPredicateConversion() { + assertPredicateConversion( + Expressions.greaterThanOrEqual( + Expressions.extract("payload", "$.age", "long"), + 30L), + VARIANT_MESSAGE_WITH_LONG, + FilterApi.gtEq( + FilterApi.longColumn("payload.typed_value.age.typed_value"), + 30L)); + } + + @Test + public void testNestedPathConversion() { + assertPredicateConversion( + Expressions.equal( + Expressions.extract("payload", "$.a.b", "string"), + "gold"), + VARIANT_MESSAGE_NESTED, + FilterApi.eq( + FilterApi.binaryColumn("payload.typed_value.a.b.typed_value"), + Binary.fromString("gold"))); + } + + @Test + public void testStructNestedVariantConversion() { + assertPredicateConversion( + Expressions.equal( + Expressions.extract("payload.info", "$.city", "string"), + "Seattle"), + VARIANT_MESSAGE_STRUCT_NESTED, + FilterApi.eq( + FilterApi.binaryColumn("payload.info.typed_value.city.typed_value"), + Binary.fromString("Seattle"))); + } + + @Test + public void testIsNullPredicateConversion() { + assertNullPredicateConversion( + Expressions.isNull( + Expressions.extract("payload", "$.tier", "string")), + VARIANT_MESSAGE, + "payload.typed_value.tier.typed_value", + true); + } + + @Test + public void testNotNullPredicateConversion() { + assertNullPredicateConversion( + Expressions.notNull( + Expressions.extract("payload", "$.tier", "string")), + VARIANT_MESSAGE, + "payload.typed_value.tier.typed_value", + false); + } + + @Test + public void testIsNullOnNestedPath() { + assertNullPredicateConversion( + Expressions.isNull( + Expressions.extract("payload", "$.a.b", "string")), + VARIANT_MESSAGE_NESTED, + "payload.typed_value.a.b.typed_value", + true); + } + + @Test + public void testNotEqualPredicateConversion() { + assertPredicateConversion( + Expressions.notEqual("payload.typed_value.tier", "gold"), + VARIANT_MESSAGE, + FilterApi.notEq( + FilterApi.binaryColumn("payload.typed_value.tier.typed_value"), + Binary.fromString("gold"))); + } + + @Test + public void testGreaterThanPredicateConversion() { + assertPredicateConversion( + Expressions.greaterThan( + Expressions.extract("payload", "$.age", "long"), + 25L), + VARIANT_MESSAGE_WITH_LONG, + FilterApi.gt( + FilterApi.longColumn("payload.typed_value.age.typed_value"), + 25L)); + } + + @Test + public void testLessThanOrEqualPredicateConversion() { + assertPredicateConversion( + Expressions.lessThanOrEqual( + Expressions.extract("payload", "$.age", "long"), + 40L), + VARIANT_MESSAGE_WITH_LONG, + FilterApi.ltEq( + FilterApi.longColumn("payload.typed_value.age.typed_value"), + 40L)); + } + + @Test + public void testInt32PredicateConversion() { + assertPredicateConversion( + Expressions.equal( + Expressions.extract("payload", "$.int_field", "int"), + 100), + VARIANT_MESSAGE_MULTI_TYPE, + FilterApi.eq( + FilterApi.intColumn("payload.typed_value.int_field.typed_value"), + 100)); + } + + @Test + public void testFloatPredicateConversion() { + assertPredicateConversion( + Expressions.greaterThan( + Expressions.extract("payload", "$.float_field", "float"), + 3.14f), + VARIANT_MESSAGE_MULTI_TYPE, + FilterApi.gt( + FilterApi.floatColumn("payload.typed_value.float_field.typed_value"), + 3.14f)); + } + + @Test + public void testDoublePredicateConversion() { + assertPredicateConversion( + Expressions.lessThan( + Expressions.extract("payload", "$.double_field", "double"), + 100.5), + VARIANT_MESSAGE_MULTI_TYPE, + FilterApi.lt( + FilterApi.doubleColumn("payload.typed_value.double_field.typed_value"), + 100.5)); + } + + @Test + public void testBooleanPredicateConversion() { + assertPredicateConversion( + Expressions.equal( + Expressions.extract("payload", "$.bool_field", "boolean"), + true), + VARIANT_MESSAGE_MULTI_TYPE, + FilterApi.eq( + FilterApi.booleanColumn("payload.typed_value.bool_field.typed_value"), + true)); + } + + @Test + public void testOrPredicateConvertible() { + // OR with both sides convertible: tier = 'gold' OR tier = 'silver' + Expression expr = Expressions.or( + Expressions.equal("payload.typed_value.tier", "gold"), + Expressions.equal("payload.typed_value.tier", "silver")); + + FilterPredicate expected = FilterApi.or( + FilterApi.eq( + FilterApi.binaryColumn("payload.typed_value.tier.typed_value"), + Binary.fromString("gold")), + FilterApi.eq( + FilterApi.binaryColumn("payload.typed_value.tier.typed_value"), + Binary.fromString("silver"))); + + assertPredicateConversion(expr, VARIANT_MESSAGE, expected); + } + + @Test + public void testOrPredicateUnconvertible() { + // OR where one side is unconvertible (unknown column) + // Should return null because OR requires both sides to be convertible + Expression expr = Expressions.or( + Expressions.equal("payload.typed_value.tier", "gold"), + Expressions.equal("payload.typed_value.unknown", "test")); + + FilterCompat.Filter filter = VariantParquetFilters.toParquetFilter(VARIANT_MESSAGE, expr); + Assert.assertNull("OR with unconvertible side should return null", filter); + } + + @Test + public void testAndPredicateConvertible() { + // AND with both sides convertible + Expression expr = Expressions.and( + Expressions.equal("payload.typed_value.tier", "gold"), + Expressions.notEqual("payload.typed_value.tier", "bronze")); + + FilterPredicate expected = FilterApi.and( + FilterApi.eq( + FilterApi.binaryColumn("payload.typed_value.tier.typed_value"), + Binary.fromString("gold")), + FilterApi.notEq( + FilterApi.binaryColumn("payload.typed_value.tier.typed_value"), + Binary.fromString("bronze"))); + + assertPredicateConversion(expr, VARIANT_MESSAGE, expected); + } + + @Test + public void testAndPredicateUnconvertible() { + // AND where one side is unconvertible (unknown column) + // Should return the convertible side (AND can drop unconvertible predicates) + Expression expr = Expressions.and( + Expressions.equal("payload.typed_value.tier", "gold"), + Expressions.equal("payload.typed_value.unknown", "test")); + + FilterPredicate expected = FilterApi.eq( + FilterApi.binaryColumn("payload.typed_value.tier.typed_value"), + Binary.fromString("gold")); + + assertPredicateConversion(expr, VARIANT_MESSAGE, expected); + } + + private void assertPredicateConversion( + Expression expr, MessageType schema, FilterPredicate expected) { + FilterCompat.Filter filter = VariantParquetFilters.toParquetFilter(schema, expr); + + Assert.assertTrue(filter instanceof FilterCompat.FilterPredicateCompat); + FilterPredicate predicate = ((FilterCompat.FilterPredicateCompat) filter).getFilterPredicate(); + + Assert.assertEquals(expected, predicate); + } + + private void assertNullPredicateConversion( + Expression expr, MessageType schema, String expectedColumnPath, boolean isNull) { + FilterPredicate expected = isNull ? + FilterApi.eq( + FilterApi.binaryColumn(expectedColumnPath), + null) : + FilterApi.notEq( + FilterApi.binaryColumn(expectedColumnPath), + null); + + assertPredicateConversion(expr, schema, expected); + } +} diff --git a/iceberg/iceberg-handler/src/test/queries/positive/variant_type_filter.q b/iceberg/iceberg-handler/src/test/queries/positive/variant_type_filter.q index c9907948ef26..7cae30779639 100644 --- a/iceberg/iceberg-handler/src/test/queries/positive/variant_type_filter.q +++ b/iceberg/iceberg-handler/src/test/queries/positive/variant_type_filter.q @@ -9,32 +9,81 @@ set hive.explain.user=false; set hive.fetch.task.conversion=none; +drop table if exists variant_filter_basic; + CREATE EXTERNAL TABLE variant_filter_basic ( - id BIGINT, - data VARIANT -) STORED BY ICEBERG tblproperties('format-version'='3'); + id INT, + data VARIANT +) STORED BY ICEBERG +TBLPROPERTIES ( + 'format-version'='3' +); INSERT INTO variant_filter_basic VALUES -(1, parse_json('{ "name": "Alice", "age": 30, "address": {"city": "Wonderland"} }')), -(2, parse_json('{ "name": "Bob", "age": 40, "address": {"city": "Builderland"} }')), -(3, parse_json('{ "name": "Charlie", "age": 28, "address": {"city": "Dreamtown"} }')); - -SELECT - try_variant_get(data, '$.name') AS name, - try_variant_get(data, '$.age', 'int') AS age, - try_variant_get(data, '$.address.city') AS city -FROM variant_filter_basic; - -SELECT - try_variant_get(data, '$.name') AS name, - try_variant_get(data, '$.age', 'int') AS age, - try_variant_get(data, '$.address.city') AS city -FROM variant_filter_basic -WHERE try_variant_get(data, '$.age', 'int') >= 30; +(1, parse_json('{"name": "John", "address": {"city": "Toronto"}}')), +(2, parse_json('{"name": "Bill", "age": 28}')), +(3, parse_json('{"name": "Henry", "age": 30, "address": {"city": "NYC"}}')); -EXPLAIN SELECT - try_variant_get(data, '$.name') AS name, - try_variant_get(data, '$.age', 'int') AS age, - try_variant_get(data, '$.address.city') AS city +-- Retrieve and verify +SELECT id, + variant_get(data, '$.name') AS name, + variant_get(data, '$.age', 'int') AS age, + variant_get(data, '$.address.city') AS city FROM variant_filter_basic -WHERE try_variant_get(data, '$.age', 'int') >= 30; \ No newline at end of file +WHERE variant_get(data, '$.age', 'int') >= 30; + +-- Test PPD on shredded variant +drop table if exists shredded_variant_ppd; + +CREATE EXTERNAL TABLE shredded_variant_ppd ( + id INT, + data VARIANT +) STORED BY ICEBERG +TBLPROPERTIES ( + 'format-version'='3', + 'variant.shredding.enabled'='true' +); + +INSERT INTO shredded_variant_ppd VALUES +(1, parse_json('{"name": "John", "address": {"city": "Toronto"}}')), +(2, parse_json('{"name": "Bill", "age": 28}')), +(3, parse_json('{"name": "Henry", "age": 30, "address": {"city": "NYC"}}')); + +-- Insert NULL variant +INSERT INTO shredded_variant_ppd (id) SELECT 4; + +-- Retrieve and verify +SELECT id, + variant_get(data, '$.name') AS name, + variant_get(data, '$.age', 'int') AS age, + variant_get(data, '$.address.city') AS city +FROM shredded_variant_ppd +WHERE variant_get(data, '$.age', 'int') >= 30; + +EXPLAIN +SELECT id, + variant_get(data, '$.name') AS name, + variant_get(data, '$.age', 'int') AS age, + variant_get(data, '$.address.city') AS city +FROM shredded_variant_ppd +WHERE variant_get(data, '$.age', 'int') >= 30; + +-- Test IS NOT NULL on entire variant column (should return rows 1-3) +SELECT id, variant_get(data, '$.name') AS name FROM shredded_variant_ppd +WHERE data IS NOT NULL; + +EXPLAIN +SELECT id, variant_get(data, '$.name') FROM shredded_variant_ppd +WHERE data IS NOT NULL; + +-- Test IS NOT NULL on field that exists in some rows (should return rows 1, 3) +SELECT id, variant_get(data, '$.name') AS name FROM shredded_variant_ppd +WHERE variant_get(data, '$.address.city') IS NOT NULL; + +EXPLAIN +SELECT id, variant_get(data, '$.name') FROM shredded_variant_ppd +WHERE variant_get(data, '$.address.city') IS NOT NULL; + +-- Test IS NULL on field that exists in some rows (should return rows 2, 4) +SELECT id, variant_get(data, '$.name') AS name FROM shredded_variant_ppd +WHERE variant_get(data, '$.address.city') IS NULL diff --git a/iceberg/iceberg-handler/src/test/queries/positive/variant_type_shredding.q b/iceberg/iceberg-handler/src/test/queries/positive/variant_type_shredding.q index df6794c431ab..adfd03f86a79 100644 --- a/iceberg/iceberg-handler/src/test/queries/positive/variant_type_shredding.q +++ b/iceberg/iceberg-handler/src/test/queries/positive/variant_type_shredding.q @@ -9,48 +9,29 @@ set hive.explain.user=false; set hive.fetch.task.conversion=none; -drop table if exists tbl_shredded_variant; +drop table if exists shredded_variant; -- Create test table -CREATE EXTERNAL TABLE tbl_shredded_variant ( - id INT, - data VARIANT -) STORED BY ICEBERG -tblproperties( - 'format-version'='3', - 'variant.shredding.enabled'='true' -); - --- Insert JSON structures -INSERT INTO tbl_shredded_variant VALUES -(1, parse_json('{"name": "John", "age": 30, "active": true}')), -(2, parse_json('{"name": "Bill", "active": false}')), -(3, parse_json('{"name": "Henry", "age": 20}')); - --- Retrieve and verify -SELECT id, try_variant_get(data, '$.name') FROM tbl_shredded_variant -WHERE variant_get(data, '$.age') > 25; - -EXPLAIN -SELECT id, try_variant_get(data, '$.name') FROM tbl_shredded_variant -WHERE variant_get(data, '$.age') > 25; - -CREATE TABLE t ( +CREATE EXTERNAL TABLE shredded_variant ( id INT, - v VARIANT -) -STORED BY ICEBERG + data VARIANT +) STORED BY ICEBERG TBLPROPERTIES ( 'format-version'='3', 'variant.shredding.enabled'='true' ); -INSERT INTO t VALUES -(1, parse_json('{"a": 1}')), -(2, parse_json('{"b": 2}')); +INSERT INTO shredded_variant VALUES +(1, parse_json('{"name": "John", "age": 30}')), +(2, parse_json('{"name": "Bill"}')), +(3, parse_json('{"name": "Henry", "address": {"city": "NYC"}}')); -SELECT - try_variant_get(v, '$.a'), - try_variant_get(v, '$.b') -FROM t -ORDER BY id; +-- Insert NULL variant +INSERT INTO shredded_variant (id) SELECT 4; + +-- Retrieve and verify +SELECT id, + variant_get(data, '$.name') AS name, + variant_get(data, '$.age', 'int') AS age, + variant_get(data, '$.address.city') AS city +FROM shredded_variant; diff --git a/iceberg/iceberg-handler/src/test/results/positive/variant_type_filter.q.out b/iceberg/iceberg-handler/src/test/results/positive/variant_type_filter.q.out index 649bfab0ce67..e10c1d5017fc 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/variant_type_filter.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/variant_type_filter.q.out @@ -1,87 +1,274 @@ +PREHOOK: query: drop table if exists variant_filter_basic +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists variant_filter_basic +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default PREHOOK: query: CREATE EXTERNAL TABLE variant_filter_basic ( - id BIGINT, - data VARIANT -) STORED BY ICEBERG tblproperties('format-version'='3') + id INT, + data VARIANT +) STORED BY ICEBERG +TBLPROPERTIES ( + 'format-version'='3' +) PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@variant_filter_basic POSTHOOK: query: CREATE EXTERNAL TABLE variant_filter_basic ( - id BIGINT, - data VARIANT -) STORED BY ICEBERG tblproperties('format-version'='3') + id INT, + data VARIANT +) STORED BY ICEBERG +TBLPROPERTIES ( + 'format-version'='3' +) POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@variant_filter_basic PREHOOK: query: INSERT INTO variant_filter_basic VALUES -(1, parse_json('{ "name": "Alice", "age": 30, "address": {"city": "Wonderland"} }')), -(2, parse_json('{ "name": "Bob", "age": 40, "address": {"city": "Builderland"} }')), -(3, parse_json('{ "name": "Charlie", "age": 28, "address": {"city": "Dreamtown"} }')) +(1, parse_json('{"name": "John", "address": {"city": "Toronto"}}')), +(2, parse_json('{"name": "Bill", "age": 28}')), +(3, parse_json('{"name": "Henry", "age": 30, "address": {"city": "NYC"}}')) PREHOOK: type: QUERY PREHOOK: Input: _dummy_database@_dummy_table PREHOOK: Output: default@variant_filter_basic POSTHOOK: query: INSERT INTO variant_filter_basic VALUES -(1, parse_json('{ "name": "Alice", "age": 30, "address": {"city": "Wonderland"} }')), -(2, parse_json('{ "name": "Bob", "age": 40, "address": {"city": "Builderland"} }')), -(3, parse_json('{ "name": "Charlie", "age": 28, "address": {"city": "Dreamtown"} }')) +(1, parse_json('{"name": "John", "address": {"city": "Toronto"}}')), +(2, parse_json('{"name": "Bill", "age": 28}')), +(3, parse_json('{"name": "Henry", "age": 30, "address": {"city": "NYC"}}')) POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@variant_filter_basic -PREHOOK: query: SELECT - try_variant_get(data, '$.name') AS name, - try_variant_get(data, '$.age', 'int') AS age, - try_variant_get(data, '$.address.city') AS city +PREHOOK: query: SELECT id, + variant_get(data, '$.name') AS name, + variant_get(data, '$.age', 'int') AS age, + variant_get(data, '$.address.city') AS city FROM variant_filter_basic +WHERE variant_get(data, '$.age', 'int') >= 30 PREHOOK: type: QUERY PREHOOK: Input: default@variant_filter_basic PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: SELECT - try_variant_get(data, '$.name') AS name, - try_variant_get(data, '$.age', 'int') AS age, - try_variant_get(data, '$.address.city') AS city +POSTHOOK: query: SELECT id, + variant_get(data, '$.name') AS name, + variant_get(data, '$.age', 'int') AS age, + variant_get(data, '$.address.city') AS city FROM variant_filter_basic +WHERE variant_get(data, '$.age', 'int') >= 30 POSTHOOK: type: QUERY POSTHOOK: Input: default@variant_filter_basic POSTHOOK: Output: hdfs://### HDFS PATH ### -Alice 30 Wonderland -Bob 40 Builderland -Charlie 28 Dreamtown -PREHOOK: query: SELECT - try_variant_get(data, '$.name') AS name, - try_variant_get(data, '$.age', 'int') AS age, - try_variant_get(data, '$.address.city') AS city -FROM variant_filter_basic -WHERE try_variant_get(data, '$.age', 'int') >= 30 +3 Henry 30 NYC +PREHOOK: query: drop table if exists shredded_variant_ppd +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists shredded_variant_ppd +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: CREATE EXTERNAL TABLE shredded_variant_ppd ( + id INT, + data VARIANT +) STORED BY ICEBERG +TBLPROPERTIES ( + 'format-version'='3', + 'variant.shredding.enabled'='true' +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@shredded_variant_ppd +POSTHOOK: query: CREATE EXTERNAL TABLE shredded_variant_ppd ( + id INT, + data VARIANT +) STORED BY ICEBERG +TBLPROPERTIES ( + 'format-version'='3', + 'variant.shredding.enabled'='true' +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@shredded_variant_ppd +PREHOOK: query: INSERT INTO shredded_variant_ppd VALUES +(1, parse_json('{"name": "John", "address": {"city": "Toronto"}}')), +(2, parse_json('{"name": "Bill", "age": 28}')), +(3, parse_json('{"name": "Henry", "age": 30, "address": {"city": "NYC"}}')) PREHOOK: type: QUERY -PREHOOK: Input: default@variant_filter_basic +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@shredded_variant_ppd +POSTHOOK: query: INSERT INTO shredded_variant_ppd VALUES +(1, parse_json('{"name": "John", "address": {"city": "Toronto"}}')), +(2, parse_json('{"name": "Bill", "age": 28}')), +(3, parse_json('{"name": "Henry", "age": 30, "address": {"city": "NYC"}}')) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@shredded_variant_ppd +PREHOOK: query: INSERT INTO shredded_variant_ppd (id) SELECT 4 +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@shredded_variant_ppd +POSTHOOK: query: INSERT INTO shredded_variant_ppd (id) SELECT 4 +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@shredded_variant_ppd +PREHOOK: query: SELECT id, + variant_get(data, '$.name') AS name, + variant_get(data, '$.age', 'int') AS age, + variant_get(data, '$.address.city') AS city +FROM shredded_variant_ppd +WHERE variant_get(data, '$.age', 'int') >= 30 +PREHOOK: type: QUERY +PREHOOK: Input: default@shredded_variant_ppd PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: SELECT - try_variant_get(data, '$.name') AS name, - try_variant_get(data, '$.age', 'int') AS age, - try_variant_get(data, '$.address.city') AS city -FROM variant_filter_basic -WHERE try_variant_get(data, '$.age', 'int') >= 30 +POSTHOOK: query: SELECT id, + variant_get(data, '$.name') AS name, + variant_get(data, '$.age', 'int') AS age, + variant_get(data, '$.address.city') AS city +FROM shredded_variant_ppd +WHERE variant_get(data, '$.age', 'int') >= 30 POSTHOOK: type: QUERY -POSTHOOK: Input: default@variant_filter_basic +POSTHOOK: Input: default@shredded_variant_ppd POSTHOOK: Output: hdfs://### HDFS PATH ### -Alice 30 Wonderland -Bob 40 Builderland -PREHOOK: query: EXPLAIN SELECT - try_variant_get(data, '$.name') AS name, - try_variant_get(data, '$.age', 'int') AS age, - try_variant_get(data, '$.address.city') AS city -FROM variant_filter_basic -WHERE try_variant_get(data, '$.age', 'int') >= 30 +3 Henry 30 NYC +PREHOOK: query: EXPLAIN +SELECT id, + variant_get(data, '$.name') AS name, + variant_get(data, '$.age', 'int') AS age, + variant_get(data, '$.address.city') AS city +FROM shredded_variant_ppd +WHERE variant_get(data, '$.age', 'int') >= 30 PREHOOK: type: QUERY -PREHOOK: Input: default@variant_filter_basic +PREHOOK: Input: default@shredded_variant_ppd PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: EXPLAIN SELECT - try_variant_get(data, '$.name') AS name, - try_variant_get(data, '$.age', 'int') AS age, - try_variant_get(data, '$.address.city') AS city -FROM variant_filter_basic -WHERE try_variant_get(data, '$.age', 'int') >= 30 +POSTHOOK: query: EXPLAIN +SELECT id, + variant_get(data, '$.name') AS name, + variant_get(data, '$.age', 'int') AS age, + variant_get(data, '$.address.city') AS city +FROM shredded_variant_ppd +WHERE variant_get(data, '$.age', 'int') >= 30 POSTHOOK: type: QUERY -POSTHOOK: Input: default@variant_filter_basic +POSTHOOK: Input: default@shredded_variant_ppd +POSTHOOK: Output: hdfs://### HDFS PATH ### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: shredded_variant_ppd + filterExpr: (variant_get(data, '$.age', 'int') >= 30) (type: boolean) + Pruned Column Paths: data.name, data.age, data.address.city + Statistics: Num rows: 4 Data size: 1360 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (variant_get(data, '$.age', 'int') >= 30) (type: boolean) + Statistics: Num rows: 1 Data size: 340 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: id (type: int), variant_get(data, '$.name') (type: string), variant_get(data, '$.age', 'int') (type: int), variant_get(data, '$.address.city') (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 1 Data size: 340 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 340 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: SELECT id, variant_get(data, '$.name') AS name FROM shredded_variant_ppd +WHERE data IS NOT NULL +PREHOOK: type: QUERY +PREHOOK: Input: default@shredded_variant_ppd +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: SELECT id, variant_get(data, '$.name') AS name FROM shredded_variant_ppd +WHERE data IS NOT NULL +POSTHOOK: type: QUERY +POSTHOOK: Input: default@shredded_variant_ppd +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 John +2 Bill +3 Henry +PREHOOK: query: EXPLAIN +SELECT id, variant_get(data, '$.name') FROM shredded_variant_ppd +WHERE data IS NOT NULL +PREHOOK: type: QUERY +PREHOOK: Input: default@shredded_variant_ppd +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: EXPLAIN +SELECT id, variant_get(data, '$.name') FROM shredded_variant_ppd +WHERE data IS NOT NULL +POSTHOOK: type: QUERY +POSTHOOK: Input: default@shredded_variant_ppd +POSTHOOK: Output: hdfs://### HDFS PATH ### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: shredded_variant_ppd + filterExpr: data is not null (type: boolean) + Statistics: Num rows: 4 Data size: 1360 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: data is not null (type: boolean) + Statistics: Num rows: 4 Data size: 1360 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: id (type: int), variant_get(data, '$.name') (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 4 Data size: 1360 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 4 Data size: 1360 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: SELECT id, variant_get(data, '$.name') AS name FROM shredded_variant_ppd +WHERE variant_get(data, '$.address.city') IS NOT NULL +PREHOOK: type: QUERY +PREHOOK: Input: default@shredded_variant_ppd +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: SELECT id, variant_get(data, '$.name') AS name FROM shredded_variant_ppd +WHERE variant_get(data, '$.address.city') IS NOT NULL +POSTHOOK: type: QUERY +POSTHOOK: Input: default@shredded_variant_ppd +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 John +3 Henry +PREHOOK: query: EXPLAIN +SELECT id, variant_get(data, '$.name') FROM shredded_variant_ppd +WHERE variant_get(data, '$.address.city') IS NOT NULL +PREHOOK: type: QUERY +PREHOOK: Input: default@shredded_variant_ppd +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: EXPLAIN +SELECT id, variant_get(data, '$.name') FROM shredded_variant_ppd +WHERE variant_get(data, '$.address.city') IS NOT NULL +POSTHOOK: type: QUERY +POSTHOOK: Input: default@shredded_variant_ppd POSTHOOK: Output: hdfs://### HDFS PATH ### STAGE DEPENDENCIES: Stage-1 is a root stage @@ -95,19 +282,20 @@ STAGE PLANS: Map 1 Map Operator Tree: TableScan - alias: variant_filter_basic - filterExpr: (try_variant_get(data, '$.age', 'int') >= 30) (type: boolean) - Statistics: Num rows: 3 Data size: 1008 Basic stats: COMPLETE Column stats: NONE + alias: shredded_variant_ppd + filterExpr: variant_get(data, '$.address.city') is not null (type: boolean) + Pruned Column Paths: data.name, data.address.city + Statistics: Num rows: 4 Data size: 1360 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: (try_variant_get(data, '$.age', 'int') >= 30) (type: boolean) - Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: NONE + predicate: variant_get(data, '$.address.city') is not null (type: boolean) + Statistics: Num rows: 4 Data size: 1360 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: try_variant_get(data, '$.name') (type: string), try_variant_get(data, '$.age', 'int') (type: int), try_variant_get(data, '$.address.city') (type: string) - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: NONE + expressions: id (type: int), variant_get(data, '$.name') (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 4 Data size: 1360 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 4 Data size: 1360 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -120,3 +308,15 @@ STAGE PLANS: Processor Tree: ListSink +PREHOOK: query: SELECT id, variant_get(data, '$.name') AS name FROM shredded_variant_ppd +WHERE variant_get(data, '$.address.city') IS NULL +PREHOOK: type: QUERY +PREHOOK: Input: default@shredded_variant_ppd +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: SELECT id, variant_get(data, '$.name') AS name FROM shredded_variant_ppd +WHERE variant_get(data, '$.address.city') IS NULL +POSTHOOK: type: QUERY +POSTHOOK: Input: default@shredded_variant_ppd +POSTHOOK: Output: hdfs://### HDFS PATH ### +2 Bill +4 NULL diff --git a/iceberg/iceberg-handler/src/test/results/positive/variant_type_shredding.q.out b/iceberg/iceberg-handler/src/test/results/positive/variant_type_shredding.q.out index f7c0910a9b89..095f7718e121 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/variant_type_shredding.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/variant_type_shredding.q.out @@ -1,155 +1,70 @@ -PREHOOK: query: drop table if exists tbl_shredded_variant +PREHOOK: query: drop table if exists shredded_variant PREHOOK: type: DROPTABLE PREHOOK: Output: database:default -POSTHOOK: query: drop table if exists tbl_shredded_variant +POSTHOOK: query: drop table if exists shredded_variant POSTHOOK: type: DROPTABLE POSTHOOK: Output: database:default -PREHOOK: query: CREATE EXTERNAL TABLE tbl_shredded_variant ( - id INT, - data VARIANT -) STORED BY ICEBERG -tblproperties( - 'format-version'='3', - 'variant.shredding.enabled'='true' -) -PREHOOK: type: CREATETABLE -PREHOOK: Output: database:default -PREHOOK: Output: default@tbl_shredded_variant -POSTHOOK: query: CREATE EXTERNAL TABLE tbl_shredded_variant ( - id INT, - data VARIANT -) STORED BY ICEBERG -tblproperties( - 'format-version'='3', - 'variant.shredding.enabled'='true' -) -POSTHOOK: type: CREATETABLE -POSTHOOK: Output: database:default -POSTHOOK: Output: default@tbl_shredded_variant -PREHOOK: query: INSERT INTO tbl_shredded_variant VALUES -(1, parse_json('{"name": "John", "age": 30, "active": true}')), -(2, parse_json('{"name": "Bill", "active": false}')), -(3, parse_json('{"name": "Henry", "age": 20}')) -PREHOOK: type: QUERY -PREHOOK: Input: _dummy_database@_dummy_table -PREHOOK: Output: default@tbl_shredded_variant -POSTHOOK: query: INSERT INTO tbl_shredded_variant VALUES -(1, parse_json('{"name": "John", "age": 30, "active": true}')), -(2, parse_json('{"name": "Bill", "active": false}')), -(3, parse_json('{"name": "Henry", "age": 20}')) -POSTHOOK: type: QUERY -POSTHOOK: Input: _dummy_database@_dummy_table -POSTHOOK: Output: default@tbl_shredded_variant -PREHOOK: query: SELECT id, try_variant_get(data, '$.name') FROM tbl_shredded_variant -WHERE variant_get(data, '$.age') > 25 -PREHOOK: type: QUERY -PREHOOK: Input: default@tbl_shredded_variant -PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: SELECT id, try_variant_get(data, '$.name') FROM tbl_shredded_variant -WHERE variant_get(data, '$.age') > 25 -POSTHOOK: type: QUERY -POSTHOOK: Input: default@tbl_shredded_variant -POSTHOOK: Output: hdfs://### HDFS PATH ### -1 John -PREHOOK: query: EXPLAIN -SELECT id, try_variant_get(data, '$.name') FROM tbl_shredded_variant -WHERE variant_get(data, '$.age') > 25 -PREHOOK: type: QUERY -PREHOOK: Input: default@tbl_shredded_variant -PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: EXPLAIN -SELECT id, try_variant_get(data, '$.name') FROM tbl_shredded_variant -WHERE variant_get(data, '$.age') > 25 -POSTHOOK: type: QUERY -POSTHOOK: Input: default@tbl_shredded_variant -POSTHOOK: Output: hdfs://### HDFS PATH ### -STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-0 depends on stages: Stage-1 - -STAGE PLANS: - Stage: Stage-1 - Tez -#### A masked pattern was here #### - Vertices: - Map 1 - Map Operator Tree: - TableScan - alias: tbl_shredded_variant - filterExpr: (UDFToDouble(variant_get(data, '$.age')) > 25.0D) (type: boolean) - Statistics: Num rows: 3 Data size: 1020 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: (UDFToDouble(variant_get(data, '$.age')) > 25.0D) (type: boolean) - Statistics: Num rows: 1 Data size: 340 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: id (type: int), try_variant_get(data, '$.name') (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 340 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 340 Basic stats: COMPLETE Column stats: NONE - table: - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - Stage: Stage-0 - Fetch Operator - limit: -1 - Processor Tree: - ListSink - -PREHOOK: query: CREATE TABLE t ( +PREHOOK: query: CREATE EXTERNAL TABLE shredded_variant ( id INT, - v VARIANT -) -STORED BY ICEBERG + data VARIANT +) STORED BY ICEBERG TBLPROPERTIES ( 'format-version'='3', 'variant.shredding.enabled'='true' ) PREHOOK: type: CREATETABLE PREHOOK: Output: database:default -PREHOOK: Output: default@t -POSTHOOK: query: CREATE TABLE t ( +PREHOOK: Output: default@shredded_variant +POSTHOOK: query: CREATE EXTERNAL TABLE shredded_variant ( id INT, - v VARIANT -) -STORED BY ICEBERG + data VARIANT +) STORED BY ICEBERG TBLPROPERTIES ( 'format-version'='3', 'variant.shredding.enabled'='true' ) POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default -POSTHOOK: Output: default@t -PREHOOK: query: INSERT INTO t VALUES -(1, parse_json('{"a": 1}')), -(2, parse_json('{"b": 2}')) +POSTHOOK: Output: default@shredded_variant +PREHOOK: query: INSERT INTO shredded_variant VALUES +(1, parse_json('{"name": "John", "age": 30}')), +(2, parse_json('{"name": "Bill"}')), +(3, parse_json('{"name": "Henry", "address": {"city": "NYC"}}')) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@shredded_variant +POSTHOOK: query: INSERT INTO shredded_variant VALUES +(1, parse_json('{"name": "John", "age": 30}')), +(2, parse_json('{"name": "Bill"}')), +(3, parse_json('{"name": "Henry", "address": {"city": "NYC"}}')) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@shredded_variant +PREHOOK: query: INSERT INTO shredded_variant (id) SELECT 4 PREHOOK: type: QUERY PREHOOK: Input: _dummy_database@_dummy_table -PREHOOK: Output: default@t -POSTHOOK: query: INSERT INTO t VALUES -(1, parse_json('{"a": 1}')), -(2, parse_json('{"b": 2}')) +PREHOOK: Output: default@shredded_variant +POSTHOOK: query: INSERT INTO shredded_variant (id) SELECT 4 POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table -POSTHOOK: Output: default@t -PREHOOK: query: SELECT - try_variant_get(v, '$.a'), - try_variant_get(v, '$.b') -FROM t -ORDER BY id +POSTHOOK: Output: default@shredded_variant +PREHOOK: query: SELECT id, + variant_get(data, '$.name') AS name, + variant_get(data, '$.age', 'int') AS age, + variant_get(data, '$.address.city') AS city +FROM shredded_variant PREHOOK: type: QUERY -PREHOOK: Input: default@t +PREHOOK: Input: default@shredded_variant PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: SELECT - try_variant_get(v, '$.a'), - try_variant_get(v, '$.b') -FROM t -ORDER BY id +POSTHOOK: query: SELECT id, + variant_get(data, '$.name') AS name, + variant_get(data, '$.age', 'int') AS age, + variant_get(data, '$.address.city') AS city +FROM shredded_variant POSTHOOK: type: QUERY -POSTHOOK: Input: default@t +POSTHOOK: Input: default@shredded_variant POSTHOOK: Output: hdfs://### HDFS PATH ### -1 NULL -NULL 2 +1 John 30 NULL +2 Bill NULL NULL +3 Henry NULL NYC +4 NULL NULL NULL diff --git a/iceberg/pom.xml b/iceberg/pom.xml index 1c5bfed17c02..6b83676f65b4 100644 --- a/iceberg/pom.xml +++ b/iceberg/pom.xml @@ -80,11 +80,6 @@ iceberg-core ${iceberg.version} - - org.apache.iceberg - iceberg-hive-metastore - ${iceberg.version} - org.apache.iceberg iceberg-data diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcCtx.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcCtx.java index c2a2fb103fc2..f55bd9718657 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcCtx.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcCtx.java @@ -37,10 +37,13 @@ import org.apache.hadoop.hive.ql.parse.ParseContext; import org.apache.hadoop.hive.ql.parse.SemanticException; import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeFieldDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; import org.apache.hadoop.hive.ql.plan.OperatorDesc; import org.apache.hadoop.hive.ql.plan.SelectDesc; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFVariantGet; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; @@ -250,15 +253,92 @@ private static void getNestedColsFromExprNodeDesc( checkListAndMap(fieldDesc, pathToRoot, p); getNestedColsFromExprNodeDesc(childDesc, p, paths); } else { - List children = desc.getChildren(); - if (children != null) { - for (ExprNodeDesc c : children) { - getNestedColsFromExprNodeDesc(c, pathToRoot, paths); + // Try variant-specific optimization first + boolean handled = false; + if (desc instanceof ExprNodeGenericFuncDesc funcDesc + && funcDesc.getGenericUDF() instanceof GenericUDFVariantGet) { + handled = tryHandleVariantGet(funcDesc, pathToRoot, paths); + } + + // If not handled (generic UDF or dynamic variant path), traverse children + if (!handled) { + List children = desc.getChildren(); + if (children != null) { + for (ExprNodeDesc c : children) { + getNestedColsFromExprNodeDesc(c, pathToRoot, paths); + } } } } } + /** + * Extracts nested column paths from variant_get expressions for projection pushdown optimization. + * + *

This enables downstream readers (e.g., ParquetVariantRecordReader) to skip reading + * unneeded shredded variant fields from storage, significantly reducing I/O. + */ + private static boolean tryHandleVariantGet( + ExprNodeGenericFuncDesc funcDesc, FieldNode pathToRoot, List paths) { + List children = funcDesc.getChildren(); + + // Semantic analyzer should enforce at least 2 arguments + if (children == null || children.size() < 2) { + throw new IllegalStateException( + "GenericUDFVariantGet should have at least 2 arguments after semantic analysis"); + } + + // If path argument is not a constant, we cannot extract it at compile time + if (!(children.get(1) instanceof ExprNodeConstantDesc)) { + return false; // Dynamic path - fallback to reading full column + } + + // Extract and normalize the JSONPath + String path = ((ExprNodeConstantDesc) children.get(1)).getValue().toString(); + if (path.startsWith("$")) { + path = path.substring(1); + } + if (path.startsWith(".")) { + path = path.substring(1); + } + if (path.isEmpty()) { + return false; // Root access - read full variant + } + + String[] segments = path.split("\\."); + + // Build FieldNode chain from leaf to root + // For $.address.city: city -> address -> (pathToRoot) + FieldNode currentTail = pathToRoot; + for (int i = segments.length - 1; i >= 0; i--) { + String seg = segments[i]; + + // Strip array index: items[0] -> items + int bracket = seg.indexOf('['); + if (bracket > 0) { + seg = seg.substring(0, bracket); + } + + if (seg.isEmpty()) { + continue; + } + + FieldNode fn = new FieldNode(seg); + if (currentTail != null) { + fn.addFieldNodes(currentTail); + } + currentTail = fn; + } + + // Attach the variant path chain to the column + if (currentTail != null) { + getNestedColsFromExprNodeDesc(children.getFirst(), currentTail, paths); + return true; + } + + return false; // All segments were empty - shouldn't happen with valid path + } + private static void checkListAndMap(ExprNodeDesc desc, FieldNode pathToRoot, FieldNode fn) { TypeInfo ti = desc.getTypeInfo();