Zouxxyy commented on code in PR #5242: URL: https://github.com/apache/paimon/pull/5242#discussion_r2034887409
########## paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/catalog/functions/BucketFunction.java: ########## @@ -0,0 +1,724 @@ +/* + * 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.paimon.spark.catalog.functions; + +import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.data.BinaryRowWriter; +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.Timestamp; +import org.apache.paimon.spark.SparkConversions; +import org.apache.paimon.table.BucketMode; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.types.ArrayType; +import org.apache.paimon.types.MapType; +import org.apache.paimon.types.RowType; +import org.apache.paimon.types.VariantType; + +import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableMap; + +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.functions.BoundFunction; +import org.apache.spark.sql.connector.catalog.functions.ScalarFunction; +import org.apache.spark.sql.connector.catalog.functions.UnboundFunction; +import org.apache.spark.sql.types.BinaryType; +import org.apache.spark.sql.types.BooleanType; +import org.apache.spark.sql.types.ByteType; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.types.DecimalType; +import org.apache.spark.sql.types.DoubleType; +import org.apache.spark.sql.types.FloatType; +import org.apache.spark.sql.types.IntegerType; +import org.apache.spark.sql.types.LongType; +import org.apache.spark.sql.types.ShortType; +import org.apache.spark.sql.types.StringType; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.types.TimestampNTZType; +import org.apache.spark.sql.types.TimestampType; +import org.apache.spark.unsafe.types.UTF8String; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.lang.reflect.InvocationTargetException; +import java.util.Arrays; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** A Spark function implementation for the Paimon bucket transform. */ +public class BucketFunction implements UnboundFunction { + private static final int NUM_BUCKETS_ORDINAL = 0; + private static final int SPARK_TIMESTAMP_PRECISION = 6; + + private static final Map<String, Class<? extends BucketGeneric>> BUCKET_FUNCTIONS; + + static { + ImmutableMap.Builder<String, Class<? extends BucketGeneric>> builder = + ImmutableMap.builder(); + builder.put("BucketBoolean", BucketBoolean.class); + builder.put("BucketByte", BucketByte.class); + builder.put("BucketShort", BucketShort.class); + builder.put("BucketInteger", BucketInteger.class); + builder.put("BucketLong", BucketLong.class); + builder.put("BucketFloat", BucketFloat.class); + builder.put("BucketDouble", BucketDouble.class); + builder.put("BucketString", BucketString.class); + builder.put("BucketDecimal", BucketDecimal.class); + builder.put("BucketTimestamp", BucketTimestamp.class); + builder.put("BucketBinary", BucketBinary.class); + + // Joint bucket fields of common types + builder.put("BucketIntegerInteger", BucketIntegerInteger.class); + builder.put("BucketIntegerLong", BucketIntegerLong.class); + builder.put("BucketIntegerString", BucketIntegerString.class); + builder.put("BucketLongInteger", BucketLongInteger.class); + builder.put("BucketLongLong", BucketLongLong.class); + builder.put("BucketLongString", BucketLongString.class); + builder.put("BucketStringInteger", BucketStringInteger.class); + builder.put("BucketStringLong", BucketStringLong.class); + builder.put("BucketStringString", BucketStringString.class); Review Comment: @zhongyujiang The functions of `fixed_bucket()` and `bucket()` are the same, so it is reasonable to keep only one (Keep `bucket` because it will be used in bucket join and so on) I tested it and found that the current implementation is indeed faster, but the e2e time is about the same. But the current implementation introduces some duplicate code, such as the conversion between spark type and paimon type, as well as some proprietary code, which I think will be difficult to maintain in the future. I suggest moving the logic of `fixed_bucket` to `bucket`, then remove `fixed_bucket`, WDYT -- 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: issues-unsubscr...@paimon.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org