ajantha-bhat commented on a change in pull request #3375: URL: https://github.com/apache/iceberg/pull/3375#discussion_r748676338
########## File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteDataFilesProcedure.java ########## @@ -0,0 +1,207 @@ +/* + * 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.spark.procedures; + +import java.util.Map; +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.RewriteDataFiles; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.SparkExpressions; +import org.apache.iceberg.spark.procedures.SparkProcedures.ProcedureBuilder; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.Expression; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.apache.spark.sql.catalyst.parser.ParserInterface; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import scala.runtime.BoxedUnit; + +/** + * A procedure that rewrites datafiles in a table. + * + * @see org.apache.iceberg.spark.actions.SparkActions#rewriteDataFiles(Table) + */ +class RewriteDataFilesProcedure extends BaseProcedure { + + private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{ + ProcedureParameter.required("table", DataTypes.StringType), + ProcedureParameter.optional("strategy", DataTypes.StringType), + ProcedureParameter.optional("sort_order", DataTypes.StringType), + ProcedureParameter.optional("options", STRING_MAP), + ProcedureParameter.optional("where", DataTypes.StringType) + }; + + // counts are not nullable since the action result is never null + private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{ + new StructField("rewritten_data_files_count", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("added_data_files_count", DataTypes.IntegerType, false, Metadata.empty()) + }); + + public static ProcedureBuilder builder() { + return new Builder<RewriteDataFilesProcedure>() { + @Override + protected RewriteDataFilesProcedure doBuild() { + return new RewriteDataFilesProcedure(tableCatalog()); + } + }; + } + + private RewriteDataFilesProcedure(TableCatalog tableCatalog) { + super(tableCatalog); + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + public StructType outputType() { + return OUTPUT_TYPE; + } + + @Override + public InternalRow[] call(InternalRow args) { + Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name()); + + return modifyIcebergTable(tableIdent, table -> { + RewriteDataFiles action = actions().rewriteDataFiles(table); + + String strategy = args.isNullAt(1) ? null : args.getString(1); + String sortOrders = args.isNullAt(2) ? null : args.getString(2); + if (sortOrders != null && (strategy == null || strategy.equalsIgnoreCase("binpack"))) { + throw new IllegalArgumentException( + "sort_order is not accepted for binpack strategy. Please configure sort strategy to use sort_order"); + } + SortOrder sortOrder = collectSortOrders(strategy, table, action, sortOrders); + action = checkAndApplyStrategy(action, strategy, sortOrder); + + if (!args.isNullAt(3)) { + action = checkAndApplyOptions(args, action); + } + + String where = args.isNullAt(4) ? null : args.getString(4); + action = checkAndApplyFilter(action, where); + + RewriteDataFiles.Result result = action.execute(); + + return toOutputRows(result); + }); + } + + private RewriteDataFiles checkAndApplyFilter(RewriteDataFiles action, String where) { + if (where != null) { + ParserInterface sqlParser = spark().sessionState().sqlParser(); + try { + Expression expression = sqlParser.parseExpression(where); + return action.filter(SparkExpressions.convert(expression)); Review comment: Hi @RussellSpitzer : I understood that filter on non-partition column can result in selecting files that 'may' have the data instead of 'must' have the data. But I don't see that restriction on existing action as well. So, to be Inline, I am not gonna block non-partition column, rather I will update the document with a warning. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
