You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by sb...@apache.org on 2023/12/05 08:27:27 UTC

(hive) branch master updated: HIVE-27918: Iceberg: Push transforms for clustering during table writes (#4908) (Sourabh Badhya reviewed by Denys Kuzmenko)

This is an automated email from the ASF dual-hosted git repository.

sbadhya pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new 8d0a38b8231 HIVE-27918: Iceberg: Push transforms for clustering during table writes (#4908) (Sourabh Badhya reviewed by Denys Kuzmenko)
8d0a38b8231 is described below

commit 8d0a38b82315aa412a9f3a1f60046d9259346819
Author: Sourabh Badhya <ia...@gmail.com>
AuthorDate: Tue Dec 5 13:57:21 2023 +0530

    HIVE-27918: Iceberg: Push transforms for clustering during table writes (#4908) (Sourabh Badhya reviewed by Denys Kuzmenko)
---
 .../iceberg/mr/hive/HiveIcebergStorageHandler.java |   34 +-
 .../mr/hive/IcebergTransformSortFunctionUtil.java  |  158 +++
 .../mr/hive/{ => udf}/GenericUDFIcebergBucket.java |    2 +-
 .../iceberg/mr/hive/udf/GenericUDFIcebergDay.java  |  129 +++
 .../iceberg/mr/hive/udf/GenericUDFIcebergHour.java |  118 +++
 .../mr/hive/udf/GenericUDFIcebergMonth.java        |  129 +++
 .../GenericUDFIcebergTruncate.java}                |  101 +-
 .../iceberg/mr/hive/udf/GenericUDFIcebergYear.java |  129 +++
 .../queries/positive/dynamic_partition_writes.q    |  100 +-
 .../delete_iceberg_copy_on_write_partitioned.q.out |   36 +-
 .../positive/dynamic_partition_writes.q.out        | 1031 ++++++++++++++++++++
 .../merge_iceberg_copy_on_write_partitioned.q.out  |   24 +-
 .../positive/merge_iceberg_partitioned_orc.q.out   |   16 +-
 .../update_iceberg_copy_on_write_partitioned.q.out |   64 +-
 .../hadoop/hive/ql/exec/FunctionRegistry.java      |   12 +-
 15 files changed, 1916 insertions(+), 167 deletions(-)

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 e3336437c77..40e4cb1cc93 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
@@ -38,7 +38,6 @@ import java.util.Set;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.atomic.AtomicInteger;
-import java.util.function.BiFunction;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 import org.apache.commons.collections.MapUtils;
@@ -76,7 +75,6 @@ import org.apache.hadoop.hive.ql.ddl.table.create.like.CreateTableLikeDesc;
 import org.apache.hadoop.hive.ql.ddl.table.misc.properties.AlterTableSetPropertiesDesc;
 import org.apache.hadoop.hive.ql.exec.ColumnInfo;
 import org.apache.hadoop.hive.ql.exec.FetchOperator;
-import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.hooks.WriteEntity;
 import org.apache.hadoop.hive.ql.io.IOConstants;
@@ -120,7 +118,6 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.StructField;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapred.InputFormat;
@@ -217,26 +214,6 @@ public class HiveIcebergStorageHandler implements HiveStoragePredicateHandler, H
 
   public static final String TABLE_DEFAULT_LOCATION = "TABLE_DEFAULT_LOCATION";
 
-  /**
-   * Function template for producing a custom sort expression function:
-   * Takes the source column index and the bucket count to creat a function where Iceberg bucket UDF is used to build
-   * the sort expression, e.g. iceberg_bucket(_col2, 5)
-   */
-  private static final transient BiFunction<Integer, Integer, Function<List<ExprNodeDesc>, ExprNodeDesc>>
-      BUCKET_SORT_EXPR =
-          (idx, bucket) -> cols -> {
-            try {
-              ExprNodeDesc icebergBucketSourceCol = cols.get(idx);
-              return ExprNodeGenericFuncDesc.newInstance(new GenericUDFIcebergBucket(), "iceberg_bucket",
-                  Lists.newArrayList(
-                      icebergBucketSourceCol,
-                      new ExprNodeConstantDesc(TypeInfoFactory.intTypeInfo, bucket)
-                  ));
-            } catch (UDFArgumentException e) {
-              throw new RuntimeException(e);
-            }
-          };
-
   private static final List<VirtualColumn> ACID_VIRTUAL_COLS = ImmutableList.of(VirtualColumn.PARTITION_SPEC_ID,
       VirtualColumn.PARTITION_HASH, VirtualColumn.FILE_PATH, VirtualColumn.ROW_POSITION);
   private static final List<FieldSchema> ACID_VIRTUAL_COLS_AS_FIELD_SCHEMA = ACID_VIRTUAL_COLS.stream()
@@ -738,14 +715,9 @@ public class HiveIcebergStorageHandler implements HiveStoragePredicateHandler, H
     int offset = (shouldOverwrite(hmsTable, writeOperation) ?
         ACID_VIRTUAL_COLS_AS_FIELD_SCHEMA : acidSelectColumns(hmsTable, writeOperation)).size();
 
-    for (TransformSpec spec : transformSpecs) {
-      int order = fieldOrderMap.get(spec.getColumnName());
-      if (TransformSpec.TransformType.BUCKET.equals(spec.getTransformType())) {
-        customSortExprs.add(BUCKET_SORT_EXPR.apply(order + offset, spec.getTransformParam().get()));
-      } else {
-        customSortExprs.add(cols -> cols.get(order + offset).clone());
-      }
-    }
+    customSortExprs.addAll(transformSpecs.stream().map(spec ->
+        IcebergTransformSortFunctionUtil.getCustomSortExprs(spec, fieldOrderMap.get(spec.getColumnName()) + offset)
+    ).collect(Collectors.toList()));
   }
 
   @Override
diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTransformSortFunctionUtil.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTransformSortFunctionUtil.java
new file mode 100644
index 00000000000..5373a4d903d
--- /dev/null
+++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTransformSortFunctionUtil.java
@@ -0,0 +1,158 @@
+/*
+ * 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.util.List;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.parse.TransformSpec;
+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.serde2.typeinfo.TypeInfoFactory;
+import org.apache.iceberg.mr.hive.udf.GenericUDFIcebergBucket;
+import org.apache.iceberg.mr.hive.udf.GenericUDFIcebergDay;
+import org.apache.iceberg.mr.hive.udf.GenericUDFIcebergHour;
+import org.apache.iceberg.mr.hive.udf.GenericUDFIcebergMonth;
+import org.apache.iceberg.mr.hive.udf.GenericUDFIcebergTruncate;
+import org.apache.iceberg.mr.hive.udf.GenericUDFIcebergYear;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+/**
+ * A utility class which provides Iceberg transform sort functions.
+ */
+public final class IcebergTransformSortFunctionUtil {
+
+  private IcebergTransformSortFunctionUtil() {
+    // not called
+  }
+
+  /**
+   * Function template for producing a custom sort expression function:
+   * Takes the source column index and the bucket count to create a function where Iceberg transform UDF is used to
+   * build the sort expression, e.g. iceberg_bucket(_col2, 5)
+   */
+  private static final transient BiFunction<Integer, Integer, Function<List<ExprNodeDesc>, ExprNodeDesc>>
+      BUCKET_SORT_EXPR =
+          (idx, bucket) -> cols -> {
+            try {
+              ExprNodeDesc icebergBucketSourceCol = cols.get(idx);
+              return ExprNodeGenericFuncDesc.newInstance(new GenericUDFIcebergBucket(), "iceberg_bucket",
+                  Lists.newArrayList(
+                      icebergBucketSourceCol,
+                      new ExprNodeConstantDesc(TypeInfoFactory.intTypeInfo, bucket)
+                  ));
+            } catch (UDFArgumentException e) {
+              throw new RuntimeException(e);
+            }
+          };
+
+  private static final transient BiFunction<Integer, Integer, Function<List<ExprNodeDesc>, ExprNodeDesc>>
+      TRUNCATE_SORT_EXPR =
+          (idx, truncateLength) -> cols -> {
+            try {
+              ExprNodeDesc icebergTruncateSourceCol = cols.get(idx);
+              return ExprNodeGenericFuncDesc.newInstance(new GenericUDFIcebergTruncate(), "iceberg_truncate",
+                  Lists.newArrayList(
+                      icebergTruncateSourceCol,
+                      new ExprNodeConstantDesc(TypeInfoFactory.intTypeInfo, truncateLength)
+                  ));
+            } catch (UDFArgumentException e) {
+              throw new RuntimeException(e);
+            }
+          };
+
+  private static final transient Function<Integer, Function<List<ExprNodeDesc>, ExprNodeDesc>>
+      YEAR_SORT_EXPR =
+          idx -> cols -> {
+            try {
+              ExprNodeDesc icebergYearSourceCol = cols.get(idx);
+              return ExprNodeGenericFuncDesc.newInstance(new GenericUDFIcebergYear(), "iceberg_year",
+                  Lists.newArrayList(
+                      icebergYearSourceCol
+                  ));
+            } catch (UDFArgumentException e) {
+              throw new RuntimeException(e);
+            }
+          };
+
+  private static final transient Function<Integer, Function<List<ExprNodeDesc>, ExprNodeDesc>>
+      MONTH_SORT_EXPR =
+          idx -> cols -> {
+            try {
+              ExprNodeDesc icebergMonthSourceCol = cols.get(idx);
+              return ExprNodeGenericFuncDesc.newInstance(new GenericUDFIcebergMonth(), "iceberg_month",
+                  Lists.newArrayList(
+                      icebergMonthSourceCol
+                  ));
+            } catch (UDFArgumentException e) {
+              throw new RuntimeException(e);
+            }
+          };
+
+  private static final transient Function<Integer, Function<List<ExprNodeDesc>, ExprNodeDesc>>
+      DAY_SORT_EXPR =
+          idx -> cols -> {
+            try {
+              ExprNodeDesc icebergDaySourceCol = cols.get(idx);
+              return ExprNodeGenericFuncDesc.newInstance(new GenericUDFIcebergDay(), "iceberg_day",
+                  Lists.newArrayList(
+                      icebergDaySourceCol
+                  ));
+            } catch (UDFArgumentException e) {
+              throw new RuntimeException(e);
+            }
+          };
+
+  private static final transient Function<Integer, Function<List<ExprNodeDesc>, ExprNodeDesc>>
+      HOUR_SORT_EXPR =
+          idx -> cols -> {
+            try {
+              ExprNodeDesc icebergHourSourceCol = cols.get(idx);
+              return ExprNodeGenericFuncDesc.newInstance(new GenericUDFIcebergHour(), "iceberg_hour",
+                  Lists.newArrayList(
+                      icebergHourSourceCol
+                  ));
+            } catch (UDFArgumentException e) {
+              throw new RuntimeException(e);
+            }
+          };
+
+  public static Function<List<ExprNodeDesc>, ExprNodeDesc> getCustomSortExprs(TransformSpec spec, int index) {
+    switch (spec.getTransformType()) {
+      case BUCKET:
+        return BUCKET_SORT_EXPR.apply(index, spec.getTransformParam().get());
+      case TRUNCATE:
+        return TRUNCATE_SORT_EXPR.apply(index, spec.getTransformParam().get());
+      case YEAR:
+        return YEAR_SORT_EXPR.apply(index);
+      case MONTH:
+        return MONTH_SORT_EXPR.apply(index);
+      case DAY:
+        return DAY_SORT_EXPR.apply(index);
+      case HOUR:
+        return HOUR_SORT_EXPR.apply(index);
+      default:
+        return cols -> cols.get(index).clone();
+    }
+  }
+
+}
diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/GenericUDFIcebergBucket.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergBucket.java
similarity index 99%
copy from iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/GenericUDFIcebergBucket.java
copy to iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergBucket.java
index 5f27319d12f..0077e6706fd 100644
--- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/GenericUDFIcebergBucket.java
+++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergBucket.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.iceberg.mr.hive;
+package org.apache.iceberg.mr.hive.udf;
 
 import java.nio.ByteBuffer;
 import java.util.function.Function;
diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergDay.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergDay.java
new file mode 100644
index 00000000000..344961222a2
--- /dev/null
+++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergDay.java
@@ -0,0 +1,129 @@
+/*
+ * 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.udf;
+
+import java.util.function.Function;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
+import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorConverter;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.iceberg.transforms.Transforms;
+import org.apache.iceberg.types.Types;
+
+/**
+ * GenericUDFIcebergDay - UDF that wraps around Iceberg's day transform function
+ */
+@Description(name = "iceberg_day",
+    value = "_FUNC_(value) - " +
+     "Returns the bucket value calculated by Iceberg bucket transform function ",
+    extended = "Example:\n  > SELECT _FUNC_('2023-01-02', 5);\n  2")
+public class GenericUDFIcebergDay extends GenericUDF {
+  private final IntWritable result = new IntWritable();
+  private transient PrimitiveObjectInspector argumentOI;
+  private transient ObjectInspectorConverters.Converter converter;
+
+  @FunctionalInterface
+  private interface UDFEvalFunction<T> {
+    void apply(T argument) throws HiveException;
+  }
+
+  private transient UDFEvalFunction<DeferredObject> evaluator;
+
+  @Override
+  public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException {
+    if (arguments.length != 1) {
+      throw new UDFArgumentLengthException(
+        "ICEBERG_DAY requires 1 arguments (value), but got " + arguments.length);
+    }
+
+    if (arguments[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+      throw new UDFArgumentException(
+        "ICEBERG_DAY first argument takes primitive types, got " + argumentOI.getTypeName());
+    }
+    argumentOI = (PrimitiveObjectInspector) arguments[0];
+
+    PrimitiveObjectInspector.PrimitiveCategory inputType = argumentOI.getPrimitiveCategory();
+    ObjectInspector outputOI;
+    switch (inputType) {
+      case DATE:
+        converter = new PrimitiveObjectInspectorConverter.DateConverter(argumentOI,
+          PrimitiveObjectInspectorFactory.writableDateObjectInspector);
+        Function<Object, Integer> dateTransform = Transforms.day().bind(Types.DateType.get());
+        evaluator = arg -> {
+          DateWritableV2 val = (DateWritableV2) converter.convert(arg.get());
+          result.set(dateTransform.apply(val.getDays()));
+        };
+        break;
+
+      case TIMESTAMP:
+        converter = new PrimitiveObjectInspectorConverter.TimestampConverter(argumentOI,
+          PrimitiveObjectInspectorFactory.writableTimestampObjectInspector);
+        Function<Object, Integer> timestampTransform = Transforms.day().bind(Types.TimestampType.withoutZone());
+        evaluator = arg -> {
+          TimestampWritableV2 val = (TimestampWritableV2) converter.convert(arg.get());
+          result.set(timestampTransform.apply(val.getNanos() / 1000L));
+        };
+        break;
+
+      case TIMESTAMPLOCALTZ:
+        converter = new PrimitiveObjectInspectorConverter.TimestampLocalTZConverter(argumentOI,
+          PrimitiveObjectInspectorFactory.writableTimestampTZObjectInspector);
+        Function<Object, Integer> timestampLocalTzTransform = Transforms.day().bind(Types.TimestampType.withZone());
+        evaluator = arg -> {
+          TimestampLocalTZWritable val = (TimestampLocalTZWritable) converter.convert(arg.get());
+          result.set(timestampLocalTzTransform.apply(val.getNanos() / 1000L));
+        };
+        break;
+
+      default:
+        throw new UDFArgumentException(
+          " ICEBERG_DAY() only takes DATE/TIMESTAMP/TIMESTAMPLOCALTZ" +
+          " types as first argument, got " + inputType);
+    }
+    outputOI = PrimitiveObjectInspectorFactory.writableIntObjectInspector;
+    return outputOI;
+  }
+
+  @Override
+  public Object evaluate(DeferredObject[] arguments) throws HiveException {
+    DeferredObject argument = arguments[0];
+    if (argument == null || argument.get() == null) {
+      return null;
+    } else {
+      evaluator.apply(argument);
+    }
+    return result;
+  }
+
+  @Override
+  public String getDisplayString(String[] children) {
+    return getStandardDisplayString("iceberg_day", children);
+  }
+
+}
diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergHour.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergHour.java
new file mode 100644
index 00000000000..9457d56e22c
--- /dev/null
+++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergHour.java
@@ -0,0 +1,118 @@
+/*
+ * 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.udf;
+
+import java.util.function.Function;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorConverter;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.iceberg.transforms.Transforms;
+import org.apache.iceberg.types.Types;
+
+/**
+ * GenericUDFIcebergHour - UDF that wraps around Iceberg's hour transform function
+ */
+@Description(name = "iceberg_hour",
+    value = "_FUNC_(value) - " +
+     "Returns the bucket value calculated by Iceberg hour transform function ",
+    extended = "Example:\n  > SELECT _FUNC_('2023-01-01 11:00:57');\n  11")
+public class GenericUDFIcebergHour extends GenericUDF {
+  private final IntWritable result = new IntWritable();
+  private transient PrimitiveObjectInspector argumentOI;
+  private transient ObjectInspectorConverters.Converter converter;
+
+  @FunctionalInterface
+  private interface UDFEvalFunction<T> {
+    void apply(T argument) throws HiveException;
+  }
+
+  private transient UDFEvalFunction<DeferredObject> evaluator;
+
+  @Override
+  public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException {
+    if (arguments.length != 1) {
+      throw new UDFArgumentLengthException(
+        "ICEBERG_YEAR requires 1 arguments (value), but got " + arguments.length);
+    }
+
+    if (arguments[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+      throw new UDFArgumentException(
+        "ICEBERG_YEAR first argument takes primitive types, got " + argumentOI.getTypeName());
+    }
+    argumentOI = (PrimitiveObjectInspector) arguments[0];
+
+    PrimitiveObjectInspector.PrimitiveCategory inputType = argumentOI.getPrimitiveCategory();
+    ObjectInspector outputOI;
+    switch (inputType) {
+      case TIMESTAMP:
+        converter = new PrimitiveObjectInspectorConverter.TimestampConverter(argumentOI,
+          PrimitiveObjectInspectorFactory.writableTimestampObjectInspector);
+        Function<Object, Integer> timestampTransform = Transforms.hour().bind(Types.TimestampType.withoutZone());
+        evaluator = arg -> {
+          TimestampWritableV2 val = (TimestampWritableV2) converter.convert(arg.get());
+          result.set(timestampTransform.apply(val.getNanos() / 1000L));
+        };
+        break;
+
+      case TIMESTAMPLOCALTZ:
+        converter = new PrimitiveObjectInspectorConverter.TimestampLocalTZConverter(argumentOI,
+          PrimitiveObjectInspectorFactory.writableTimestampTZObjectInspector);
+        Function<Object, Integer> timestampLocalTzTransform = Transforms.hour().bind(Types.TimestampType.withZone());
+        evaluator = arg -> {
+          TimestampLocalTZWritable val = (TimestampLocalTZWritable) converter.convert(arg.get());
+          result.set(timestampLocalTzTransform.apply(val.getNanos() / 1000L));
+        };
+        break;
+
+      default:
+        throw new UDFArgumentException(
+          " ICEBERG_HOUR() only takes TIMESTAMP/TIMESTAMPLOCALTZ" +
+          " types as first argument, got " + inputType);
+    }
+    outputOI = PrimitiveObjectInspectorFactory.writableIntObjectInspector;
+    return outputOI;
+  }
+
+  @Override
+  public Object evaluate(DeferredObject[] arguments) throws HiveException {
+    DeferredObject argument = arguments[0];
+    if (argument == null || argument.get() == null) {
+      return null;
+    } else {
+      evaluator.apply(argument);
+    }
+    return result;
+  }
+
+  @Override
+  public String getDisplayString(String[] children) {
+    return getStandardDisplayString("iceberg_hour", children);
+  }
+
+}
diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergMonth.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergMonth.java
new file mode 100644
index 00000000000..196f5dc6d55
--- /dev/null
+++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergMonth.java
@@ -0,0 +1,129 @@
+/*
+ * 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.udf;
+
+import java.util.function.Function;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
+import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorConverter;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.iceberg.transforms.Transforms;
+import org.apache.iceberg.types.Types;
+
+/**
+ * GenericUDFIcebergMonth - UDF that wraps around Iceberg's month transform function
+ */
+@Description(name = "iceberg_month",
+    value = "_FUNC_(value) - " +
+     "Returns the bucket value calculated by Iceberg month transform function ",
+    extended = "Example:\n  > SELECT _FUNC_('2023-01-01');\n  1")
+public class GenericUDFIcebergMonth extends GenericUDF {
+  private final IntWritable result = new IntWritable();
+  private transient PrimitiveObjectInspector argumentOI;
+  private transient ObjectInspectorConverters.Converter converter;
+
+  @FunctionalInterface
+  private interface UDFEvalFunction<T> {
+    void apply(T argument) throws HiveException;
+  }
+
+  private transient UDFEvalFunction<DeferredObject> evaluator;
+
+  @Override
+  public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException {
+    if (arguments.length != 1) {
+      throw new UDFArgumentLengthException(
+        "ICEBERG_MONTH requires 1 arguments (value), but got " + arguments.length);
+    }
+
+    if (arguments[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+      throw new UDFArgumentException(
+        "ICEBERG_MONTH first argument takes primitive types, got " + argumentOI.getTypeName());
+    }
+    argumentOI = (PrimitiveObjectInspector) arguments[0];
+
+    PrimitiveObjectInspector.PrimitiveCategory inputType = argumentOI.getPrimitiveCategory();
+    ObjectInspector outputOI;
+    switch (inputType) {
+      case DATE:
+        converter = new PrimitiveObjectInspectorConverter.DateConverter(argumentOI,
+          PrimitiveObjectInspectorFactory.writableDateObjectInspector);
+        Function<Object, Integer> dateTransform = Transforms.month().bind(Types.DateType.get());
+        evaluator = arg -> {
+          DateWritableV2 val = (DateWritableV2) converter.convert(arg.get());
+          result.set(dateTransform.apply(val.getDays()));
+        };
+        break;
+
+      case TIMESTAMP:
+        converter = new PrimitiveObjectInspectorConverter.TimestampConverter(argumentOI,
+          PrimitiveObjectInspectorFactory.writableTimestampObjectInspector);
+        Function<Object, Integer> timestampTransform = Transforms.month().bind(Types.TimestampType.withoutZone());
+        evaluator = arg -> {
+          TimestampWritableV2 val = (TimestampWritableV2) converter.convert(arg.get());
+          result.set(timestampTransform.apply(val.getNanos() / 1000L));
+        };
+        break;
+
+      case TIMESTAMPLOCALTZ:
+        converter = new PrimitiveObjectInspectorConverter.TimestampLocalTZConverter(argumentOI,
+          PrimitiveObjectInspectorFactory.writableTimestampTZObjectInspector);
+        Function<Object, Integer> timestampLocalTzTransform = Transforms.month().bind(Types.TimestampType.withZone());
+        evaluator = arg -> {
+          TimestampLocalTZWritable val = (TimestampLocalTZWritable) converter.convert(arg.get());
+          result.set(timestampLocalTzTransform.apply(val.getNanos() / 1000L));
+        };
+        break;
+
+      default:
+        throw new UDFArgumentException(
+          " ICEBERG_MONTH() only takes DATE/TIMESTAMP/TIMESTAMPLOCALTZ" +
+          " types as first argument, got " + inputType);
+    }
+    outputOI = PrimitiveObjectInspectorFactory.writableIntObjectInspector;
+    return outputOI;
+  }
+
+  @Override
+  public Object evaluate(DeferredObject[] arguments) throws HiveException {
+    DeferredObject argument = arguments[0];
+    if (argument == null || argument.get() == null) {
+      return null;
+    } else {
+      evaluator.apply(argument);
+    }
+    return result;
+  }
+
+  @Override
+  public String getDisplayString(String[] children) {
+    return getStandardDisplayString("iceberg_month", children);
+  }
+
+}
diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/GenericUDFIcebergBucket.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergTruncate.java
similarity index 59%
rename from iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/GenericUDFIcebergBucket.java
rename to iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergTruncate.java
index 5f27319d12f..bc0b63fefe8 100644
--- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/GenericUDFIcebergBucket.java
+++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergTruncate.java
@@ -16,16 +16,15 @@
  * limitations under the License.
  */
 
-package org.apache.iceberg.mr.hive;
+package org.apache.iceberg.mr.hive.udf;
 
-import java.nio.ByteBuffer;
+import java.math.BigDecimal;
 import java.util.function.Function;
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
-import org.apache.hadoop.hive.serde2.io.DateWritableV2;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters;
@@ -35,25 +34,25 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantIntObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
-import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
 import org.apache.iceberg.transforms.Transforms;
 import org.apache.iceberg.types.Type;
 import org.apache.iceberg.types.Types;
 
 /**
- * GenericUDFIcebergBucket - UDF that wraps around Iceberg's bucket transform function
+ * GenericUDFIcebergTruncate - UDF that wraps around Iceberg's truncate transform function
  */
-@Description(name = "iceberg_bucket",
-    value = "_FUNC_(value, bucketCount) - " +
-        "Returns the bucket value calculated by Iceberg bucket transform function ",
-    extended = "Example:\n  > SELECT _FUNC_('A bucket full of ice!', 5);\n  4")
-public class GenericUDFIcebergBucket extends GenericUDF {
-  private final IntWritable result = new IntWritable();
-  private int numBuckets = -1;
+@Description(name = "iceberg_truncate",
+    value = "_FUNC_(value, truncateLength) - " +
+     "Returns the bucket value calculated by Iceberg bucket transform function ",
+    extended = "Example:\n  > SELECT _FUNC_('abcdefgh', 5);\n  abcde")
+public class GenericUDFIcebergTruncate extends GenericUDF {
+  private final Text result = new Text();
+  private int truncateLength = 0;
   private transient PrimitiveObjectInspector argumentOI;
   private transient ObjectInspectorConverters.Converter converter;
 
@@ -68,14 +67,14 @@ public class GenericUDFIcebergBucket extends GenericUDF {
   public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException {
     if (arguments.length != 2) {
       throw new UDFArgumentLengthException(
-          "ICEBERG_BUCKET requires 2 arguments (value, bucketCount), but got " + arguments.length);
+        "ICEBERG_BUCKET requires 2 arguments (value, bucketCount), but got " + arguments.length);
     }
 
-    numBuckets = getNumBuckets(arguments[1]);
+    truncateLength = getTruncateLength(arguments[1]);
 
     if (arguments[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
       throw new UDFArgumentException(
-          "ICEBERG_BUCKET first argument takes primitive types, got " + argumentOI.getTypeName());
+        "ICEBERG_BUCKET first argument takes primitive types, got " + argumentOI.getTypeName());
     }
     argumentOI = (PrimitiveObjectInspector) arguments[0];
 
@@ -86,41 +85,30 @@ public class GenericUDFIcebergBucket extends GenericUDF {
       case VARCHAR:
       case STRING:
         converter = new PrimitiveObjectInspectorConverter.StringConverter(argumentOI);
-        Function<Object, Integer> stringTransform = Transforms.bucket(numBuckets).bind(Types.StringType.get());
+        Function<Object, Object> stringTransform = Transforms.truncate(truncateLength).bind(Types.StringType.get());
         evaluator = arg -> {
           String val = (String) converter.convert(arg.get());
-          result.set(stringTransform.apply(val));
-        };
-        break;
-
-      case BINARY:
-        converter = new PrimitiveObjectInspectorConverter.BinaryConverter(argumentOI,
-            PrimitiveObjectInspectorFactory.writableBinaryObjectInspector);
-        Function<Object, Integer> byteBufferTransform = Transforms.bucket(numBuckets).bind(Types.BinaryType.get());
-        evaluator = arg -> {
-          BytesWritable val = (BytesWritable) converter.convert(arg.get());
-          ByteBuffer byteBuffer = ByteBuffer.wrap(val.getBytes(), 0, val.getLength());
-          result.set(byteBufferTransform.apply(byteBuffer));
+          result.set(String.valueOf(stringTransform.apply(val)));
         };
         break;
 
       case INT:
         converter = new PrimitiveObjectInspectorConverter.IntConverter(argumentOI,
-            PrimitiveObjectInspectorFactory.writableIntObjectInspector);
-        Function<Object, Integer> intTransform = Transforms.bucket(numBuckets).bind(Types.IntegerType.get());
+          PrimitiveObjectInspectorFactory.writableIntObjectInspector);
+        Function<Object, Object> intTransform = Transforms.truncate(truncateLength).bind(Types.IntegerType.get());
         evaluator = arg -> {
           IntWritable val = (IntWritable) converter.convert(arg.get());
-          result.set(intTransform.apply(val.get()));
+          result.set(String.valueOf(intTransform.apply(val.get())));
         };
         break;
 
       case LONG:
         converter = new PrimitiveObjectInspectorConverter.LongConverter(argumentOI,
-            PrimitiveObjectInspectorFactory.writableLongObjectInspector);
-        Function<Object, Integer> longTransform = Transforms.bucket(numBuckets).bind(Types.LongType.get());
+          PrimitiveObjectInspectorFactory.writableLongObjectInspector);
+        Function<Object, Object> longTransform = Transforms.truncate(truncateLength).bind(Types.LongType.get());
         evaluator = arg -> {
           LongWritable val = (LongWritable) converter.convert(arg.get());
-          result.set(longTransform.apply(val.get()));
+          result.set(String.valueOf(longTransform.apply(val.get())));
         };
         break;
 
@@ -130,57 +118,46 @@ public class GenericUDFIcebergBucket extends GenericUDF {
             decimalTypeInfo.getScale());
 
         converter = new PrimitiveObjectInspectorConverter.HiveDecimalConverter(argumentOI,
-            PrimitiveObjectInspectorFactory.writableHiveDecimalObjectInspector);
-        Function<Object, Integer> bigDecimalTransform = Transforms.bucket(numBuckets).bind(decimalIcebergType);
+          PrimitiveObjectInspectorFactory.writableHiveDecimalObjectInspector);
+        Function<Object, Object> bigDecimalTransform = Transforms.truncate(truncateLength).bind(decimalIcebergType);
         evaluator = arg -> {
           HiveDecimalWritable val = (HiveDecimalWritable) converter.convert(arg.get());
-          result.set(bigDecimalTransform.apply(val.getHiveDecimal().bigDecimalValue()));
+          result.set(((BigDecimal) bigDecimalTransform.apply(val.getHiveDecimal().bigDecimalValue())).toPlainString());
         };
         break;
 
       case FLOAT:
         converter = new PrimitiveObjectInspectorConverter.FloatConverter(argumentOI,
-            PrimitiveObjectInspectorFactory.writableFloatObjectInspector);
-        Function<Object, Integer> floatTransform = Transforms.bucket(numBuckets).bind(Types.FloatType.get());
+          PrimitiveObjectInspectorFactory.writableFloatObjectInspector);
+        Function<Object, Object> floatTransform = Transforms.truncate(truncateLength).bind(Types.FloatType.get());
         evaluator = arg -> {
           FloatWritable val = (FloatWritable) converter.convert(arg.get());
-          result.set(floatTransform.apply(val.get()));
+          result.set(String.valueOf(floatTransform.apply(val.get())));
         };
         break;
 
       case DOUBLE:
         converter = new PrimitiveObjectInspectorConverter.DoubleConverter(argumentOI,
-            PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
-        Function<Object, Integer> doubleTransform = Transforms.bucket(numBuckets).bind(Types.DoubleType.get());
+          PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
+        Function<Object, Object> doubleTransform = Transforms.truncate(truncateLength).bind(Types.DoubleType.get());
         evaluator = arg -> {
           DoubleWritable val = (DoubleWritable) converter.convert(arg.get());
-          result.set(doubleTransform.apply(val.get()));
-        };
-        break;
-
-      case DATE:
-        converter = new PrimitiveObjectInspectorConverter.DateConverter(argumentOI,
-          PrimitiveObjectInspectorFactory.writableDateObjectInspector);
-        Function<Object, Integer> dateTransform = Transforms.bucket(numBuckets).bind(Types.DateType.get());
-        evaluator = arg -> {
-          DateWritableV2 val = (DateWritableV2) converter.convert(arg.get());
-          result.set(dateTransform.apply(val.getDays()));
+          result.set(String.valueOf(doubleTransform.apply(val.get())));
         };
         break;
 
       default:
         throw new UDFArgumentException(
-            " ICEBERG_BUCKET() only takes STRING/CHAR/VARCHAR/BINARY/INT/LONG/DECIMAL/FLOAT/DOUBLE/DATE" +
-                " types as first argument, got " + inputType);
+          " ICEBERG_TRUNCATE() only takes STRING/CHAR/VARCHAR/INT/LONG/DECIMAL/FLOAT/DOUBLE" +
+          " types as first argument, got " + inputType);
     }
-
-    outputOI = PrimitiveObjectInspectorFactory.writableIntObjectInspector;
+    outputOI = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
     return outputOI;
   }
 
-  private static int getNumBuckets(ObjectInspector arg) throws UDFArgumentException {
-    UDFArgumentException udfArgumentException = new UDFArgumentException("ICEBERG_BUCKET() second argument can only " +
-        "take an int type, but got " + arg.getTypeName());
+  private static int getTruncateLength(ObjectInspector arg) throws UDFArgumentException {
+    UDFArgumentException udfArgumentException = new UDFArgumentException("ICEBERG_TRUNCATE() second argument can " +
+        " only take an int type, but got " + arg.getTypeName());
     if (arg.getCategory() != ObjectInspector.Category.PRIMITIVE) {
       throw udfArgumentException;
     }
@@ -193,19 +170,17 @@ public class GenericUDFIcebergBucket extends GenericUDF {
 
   @Override
   public Object evaluate(DeferredObject[] arguments) throws HiveException {
-
     DeferredObject argument = arguments[0];
     if (argument == null || argument.get() == null) {
       return null;
     } else {
       evaluator.apply(argument);
     }
-
     return result;
   }
 
   @Override
   public String getDisplayString(String[] children) {
-    return getStandardDisplayString("iceberg_bucket", children);
+    return getStandardDisplayString("iceberg_truncate", children);
   }
 }
diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergYear.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergYear.java
new file mode 100644
index 00000000000..19754bfc01d
--- /dev/null
+++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergYear.java
@@ -0,0 +1,129 @@
+/*
+ * 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.udf;
+
+import java.util.function.Function;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
+import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorConverter;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.iceberg.transforms.Transforms;
+import org.apache.iceberg.types.Types;
+
+/**
+ * GenericUDFIcebergYear - UDF that wraps around Iceberg's year transform function
+ */
+@Description(name = "iceberg_year",
+    value = "_FUNC_(value) - " +
+     "Returns the bucket value calculated by Iceberg year transform function ",
+    extended = "Example:\n  > SELECT _FUNC_('2023-01-01');\n  2023")
+public class GenericUDFIcebergYear extends GenericUDF {
+  private final IntWritable result = new IntWritable();
+  private transient PrimitiveObjectInspector argumentOI;
+  private transient ObjectInspectorConverters.Converter converter;
+
+  @FunctionalInterface
+  private interface UDFEvalFunction<T> {
+    void apply(T argument) throws HiveException;
+  }
+
+  private transient UDFEvalFunction<DeferredObject> evaluator;
+
+  @Override
+  public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException {
+    if (arguments.length != 1) {
+      throw new UDFArgumentLengthException(
+        "ICEBERG_YEAR requires 1 arguments (value), but got " + arguments.length);
+    }
+
+    if (arguments[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+      throw new UDFArgumentException(
+        "ICEBERG_YEAR first argument takes primitive types, got " + argumentOI.getTypeName());
+    }
+    argumentOI = (PrimitiveObjectInspector) arguments[0];
+
+    PrimitiveObjectInspector.PrimitiveCategory inputType = argumentOI.getPrimitiveCategory();
+    ObjectInspector outputOI;
+    switch (inputType) {
+      case DATE:
+        converter = new PrimitiveObjectInspectorConverter.DateConverter(argumentOI,
+          PrimitiveObjectInspectorFactory.writableDateObjectInspector);
+        Function<Object, Integer> dateTransform = Transforms.year().bind(Types.DateType.get());
+        evaluator = arg -> {
+          DateWritableV2 val = (DateWritableV2) converter.convert(arg.get());
+          result.set(dateTransform.apply(val.getDays()));
+        };
+        break;
+
+      case TIMESTAMP:
+        converter = new PrimitiveObjectInspectorConverter.TimestampConverter(argumentOI,
+          PrimitiveObjectInspectorFactory.writableTimestampObjectInspector);
+        Function<Object, Integer> timestampTransform = Transforms.year().bind(Types.TimestampType.withoutZone());
+        evaluator = arg -> {
+          TimestampWritableV2 val = (TimestampWritableV2) converter.convert(arg.get());
+          result.set(timestampTransform.apply(val.getNanos() / 1000L));
+        };
+        break;
+
+      case TIMESTAMPLOCALTZ:
+        converter = new PrimitiveObjectInspectorConverter.TimestampLocalTZConverter(argumentOI,
+          PrimitiveObjectInspectorFactory.writableTimestampTZObjectInspector);
+        Function<Object, Integer> timestampLocalTzTransform = Transforms.year().bind(Types.TimestampType.withZone());
+        evaluator = arg -> {
+          TimestampLocalTZWritable val = (TimestampLocalTZWritable) converter.convert(arg.get());
+          result.set(timestampLocalTzTransform.apply(val.getNanos() / 1000L));
+        };
+        break;
+
+      default:
+        throw new UDFArgumentException(
+          " ICEBERG_YEAR() only takes DATE/TIMESTAMP/TIMESTAMPLOCALTZ" +
+          " types as first argument, got " + inputType);
+    }
+    outputOI = PrimitiveObjectInspectorFactory.writableIntObjectInspector;
+    return outputOI;
+  }
+
+  @Override
+  public Object evaluate(DeferredObject[] arguments) throws HiveException {
+    DeferredObject argument = arguments[0];
+    if (argument == null || argument.get() == null) {
+      return null;
+    } else {
+      evaluator.apply(argument);
+    }
+    return result;
+  }
+
+  @Override
+  public String getDisplayString(String[] children) {
+    return getStandardDisplayString("iceberg_year", children);
+  }
+
+}
diff --git a/iceberg/iceberg-handler/src/test/queries/positive/dynamic_partition_writes.q b/iceberg/iceberg-handler/src/test/queries/positive/dynamic_partition_writes.q
index 85063e2b095..54e46a2a260 100644
--- a/iceberg/iceberg-handler/src/test/queries/positive/dynamic_partition_writes.q
+++ b/iceberg/iceberg-handler/src/test/queries/positive/dynamic_partition_writes.q
@@ -8,6 +8,16 @@ drop table if exists tbl_target_identity;
 drop table if exists tbl_target_bucket;
 drop table if exists tbl_target_mixed;
 drop table if exists tbl_bucket_date;
+drop table if exists tbl_target_truncate_str;
+drop table if exists tbl_target_truncate_int;
+drop table if exists tbl_target_truncate_bigint;
+drop table if exists tbl_year_date;
+drop table if exists tbl_year_timestamp;
+drop table if exists tbl_month_date;
+drop table if exists tbl_month_timestamp;
+drop table if exists tbl_day_date;
+drop table if exists tbl_day_timestamp;
+drop table if exists tbl_hour_timestamp;
 
 create external table tbl_src (a int, b string, c bigint) stored by iceberg stored as orc;
 insert into tbl_src values (1, 'EUR', 10), (2, 'EUR', 10), (3, 'USD', 11), (4, 'EUR', 12), (5, 'HUF', 30), (6, 'USD', 10), (7, 'USD', 100), (8, 'PLN', 20), (9, 'PLN', 11), (10, 'CZK', 5), (12, NULL, NULL);
@@ -53,4 +63,92 @@ tblproperties ('parquet.compression'='snappy','format-version'='2');
 insert into tbl_bucket_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018);
 update tbl_bucket_date set date_time_date = '2018-07-02' where date_time_date = '2018-07-03'; 
     
-select count(*) from tbl_bucket_date where date_time_date = '2018-07-02';  
+select count(*) from tbl_bucket_date where date_time_date = '2018-07-02';
+
+--truncate case - should invoke GenericUDFIcebergTruncate to truncate the column value and use for clustering and sorting
+create external table tbl_target_truncate_str (a int, ccy string) partitioned by spec (truncate(2, ccy)) stored by iceberg stored as orc;
+explain insert into table tbl_target_truncate_str select a, b from tbl_src;
+insert into table tbl_target_truncate_str select a, b from tbl_src;
+select * from tbl_target_truncate_str order by a, ccy;
+
+create external table tbl_target_truncate_int (id int, ccy string) partitioned by spec (truncate(2, id)) stored by iceberg stored as orc;
+explain insert into table tbl_target_truncate_int select a, b from tbl_src;
+insert into table tbl_target_truncate_int select a, b from tbl_src;
+select * from tbl_target_truncate_int order by id, ccy;
+
+create external table tbl_target_truncate_bigint (a int, ccy bigint) partitioned by spec (truncate(2, ccy)) stored by iceberg stored as orc;
+explain insert into table tbl_target_truncate_bigint select a, c from tbl_src;
+insert into table tbl_target_truncate_bigint select a, c from tbl_src;
+select * from tbl_target_truncate_bigint order by a, ccy;
+
+create external table tbl_target_truncate_decimal (a int, b string, ccy decimal(10,6)) partitioned by spec (truncate(2, b), truncate(3, ccy)) stored by iceberg stored as orc;
+explain insert into table tbl_target_truncate_decimal select a, b, 1.567894 from tbl_src;
+insert into table tbl_target_truncate_decimal select a, b, 1.567894 from tbl_src;
+select * from tbl_target_truncate_decimal order by a, b;
+
+--year case - should invoke GenericUDFIcebergYear to convert the date/timestamp value to year and use for clustering and sorting
+create external table tbl_year_date (id string, date_time_date date, year_partition int)
+    partitioned by spec (year_partition, year(date_time_date))
+stored by iceberg stored as parquet
+tblproperties ('parquet.compression'='snappy','format-version'='2');
+
+explain insert into tbl_year_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018);
+insert into tbl_year_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018);
+select * from tbl_year_date order by id, date_time_date;
+
+create external table tbl_year_timestamp (id string, date_time_timestamp timestamp, year_partition int)
+    partitioned by spec (year_partition, year(date_time_timestamp))
+stored by iceberg stored as parquet
+tblproperties ('parquet.compression'='snappy','format-version'='2');
+
+explain insert into tbl_year_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018);
+insert into tbl_year_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018);
+select * from tbl_year_timestamp order by id, date_time_timestamp;
+
+--month case - should invoke GenericUDFIcebergMonth to convert the date/timestamp value to month and use for clustering and sorting
+create external table tbl_month_date (id string, date_time_date date, year_partition int)
+    partitioned by spec (year_partition, month(date_time_date))
+stored by iceberg stored as parquet
+tblproperties ('parquet.compression'='snappy','format-version'='2');
+
+explain insert into tbl_month_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018);
+insert into tbl_month_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018);
+select * from tbl_month_date order by id, date_time_date;
+
+create external table tbl_month_timestamp (id string, date_time_timestamp timestamp, year_partition int)
+    partitioned by spec (year_partition, month(date_time_timestamp))
+stored by iceberg stored as parquet
+tblproperties ('parquet.compression'='snappy','format-version'='2');
+
+explain insert into tbl_month_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018);
+insert into tbl_month_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018);
+select * from tbl_month_timestamp order by id, date_time_timestamp;
+
+--day case - should invoke GenericUDFIcebergMonth to convert the date/timestamp value to day and use for clustering and sorting
+create external table tbl_day_date (id string, date_time_date date, year_partition int)
+    partitioned by spec (year_partition, day(date_time_date))
+stored by iceberg stored as parquet
+tblproperties ('parquet.compression'='snappy','format-version'='2');
+
+explain insert into tbl_day_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018);
+insert into tbl_day_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018);
+select * from tbl_day_date order by id, date_time_date;
+
+create external table tbl_day_timestamp (id string, date_time_timestamp timestamp, year_partition int)
+    partitioned by spec (year_partition, day(date_time_timestamp))
+stored by iceberg stored as parquet
+tblproperties ('parquet.compression'='snappy','format-version'='2');
+
+explain insert into tbl_day_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018);
+insert into tbl_day_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018);
+select * from tbl_day_timestamp order by id, date_time_timestamp;
+
+--hour case - should invoke GenericUDFIcebergMonth to convert the date/timestamp value to day and use for clustering and sorting
+create external table tbl_hour_timestamp (id string, date_time_timestamp timestamp, year_partition int)
+    partitioned by spec (year_partition, hour(date_time_timestamp))
+stored by iceberg stored as parquet
+tblproperties ('parquet.compression'='snappy','format-version'='2');
+
+explain insert into tbl_hour_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018);
+insert into tbl_hour_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018);
+select * from tbl_hour_timestamp order by id, date_time_timestamp;
\ No newline at end of file
diff --git a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_partitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_partitioned.q.out
index 6206f479aac..e5fe8e90db7 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_partitioned.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_partitioned.q.out
@@ -432,10 +432,10 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                       Statistics: Num rows: 3 Data size: 903 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                        key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                         null sort order: aa
                         sort order: ++
-                        Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                        Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                         Statistics: Num rows: 7 Data size: 2107 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int)
         Reducer 19 
@@ -663,18 +663,18 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                       Statistics: Num rows: 4 Data size: 1204 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                        key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                         null sort order: aa
                         sort order: ++
-                        Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                        Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                         Statistics: Num rows: 7 Data size: 2107 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int)
         Reducer 8 
             Execution mode: vectorized
             Reduce Operator Tree:
               Select Operator
-                expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY._col5 (type: string)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), _col5
+                expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY.iceberg_truncate(_col5, 3) (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), iceberg_truncate(_col5, 3)
                 File Output Operator
                   compressed: false
                   Dp Sort State: PARTITION_SORTED
@@ -1034,18 +1034,18 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                   Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
-                    key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                    key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                     null sort order: aa
                     sort order: ++
-                    Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                    Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                     Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int)
         Reducer 6 
             Execution mode: vectorized
             Reduce Operator Tree:
               Select Operator
-                expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY._col5 (type: string)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), _col5
+                expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY.iceberg_truncate(_col5, 3) (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), iceberg_truncate(_col5, 3)
                 File Output Operator
                   compressed: false
                   Dp Sort State: PARTITION_SORTED
@@ -1106,10 +1106,10 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                       Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                        key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                         null sort order: aa
                         sort order: ++
-                        Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                        Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                         Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int)
         Reducer 9 
@@ -1486,18 +1486,18 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                   Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
-                    key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                    key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                     null sort order: aa
                     sort order: ++
-                    Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                    Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                     Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int)
         Reducer 6 
             Execution mode: vectorized
             Reduce Operator Tree:
               Select Operator
-                expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY._col5 (type: string)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), _col5
+                expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY.iceberg_truncate(_col5, 3) (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), iceberg_truncate(_col5, 3)
                 File Output Operator
                   compressed: false
                   Dp Sort State: PARTITION_SORTED
@@ -1558,10 +1558,10 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                       Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                        key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                         null sort order: aa
                         sort order: ++
-                        Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                        Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                         Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int)
         Reducer 9 
diff --git a/iceberg/iceberg-handler/src/test/results/positive/dynamic_partition_writes.q.out b/iceberg/iceberg-handler/src/test/results/positive/dynamic_partition_writes.q.out
index a05ebf9af73..b7690c5579f 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/dynamic_partition_writes.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/dynamic_partition_writes.q.out
@@ -28,6 +28,66 @@ PREHOOK: Output: database:default
 POSTHOOK: query: drop table if exists tbl_bucket_date
 POSTHOOK: type: DROPTABLE
 POSTHOOK: Output: database:default
+PREHOOK: query: drop table if exists tbl_target_truncate_str
+PREHOOK: type: DROPTABLE
+PREHOOK: Output: database:default
+POSTHOOK: query: drop table if exists tbl_target_truncate_str
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Output: database:default
+PREHOOK: query: drop table if exists tbl_target_truncate_int
+PREHOOK: type: DROPTABLE
+PREHOOK: Output: database:default
+POSTHOOK: query: drop table if exists tbl_target_truncate_int
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Output: database:default
+PREHOOK: query: drop table if exists tbl_target_truncate_bigint
+PREHOOK: type: DROPTABLE
+PREHOOK: Output: database:default
+POSTHOOK: query: drop table if exists tbl_target_truncate_bigint
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Output: database:default
+PREHOOK: query: drop table if exists tbl_year_date
+PREHOOK: type: DROPTABLE
+PREHOOK: Output: database:default
+POSTHOOK: query: drop table if exists tbl_year_date
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Output: database:default
+PREHOOK: query: drop table if exists tbl_year_timestamp
+PREHOOK: type: DROPTABLE
+PREHOOK: Output: database:default
+POSTHOOK: query: drop table if exists tbl_year_timestamp
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Output: database:default
+PREHOOK: query: drop table if exists tbl_month_date
+PREHOOK: type: DROPTABLE
+PREHOOK: Output: database:default
+POSTHOOK: query: drop table if exists tbl_month_date
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Output: database:default
+PREHOOK: query: drop table if exists tbl_month_timestamp
+PREHOOK: type: DROPTABLE
+PREHOOK: Output: database:default
+POSTHOOK: query: drop table if exists tbl_month_timestamp
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Output: database:default
+PREHOOK: query: drop table if exists tbl_day_date
+PREHOOK: type: DROPTABLE
+PREHOOK: Output: database:default
+POSTHOOK: query: drop table if exists tbl_day_date
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Output: database:default
+PREHOOK: query: drop table if exists tbl_day_timestamp
+PREHOOK: type: DROPTABLE
+PREHOOK: Output: database:default
+POSTHOOK: query: drop table if exists tbl_day_timestamp
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Output: database:default
+PREHOOK: query: drop table if exists tbl_hour_timestamp
+PREHOOK: type: DROPTABLE
+PREHOOK: Output: database:default
+POSTHOOK: query: drop table if exists tbl_hour_timestamp
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Output: database:default
 PREHOOK: query: create external table tbl_src (a int, b string, c bigint) stored by iceberg stored as orc
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
@@ -606,3 +666,974 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@tbl_bucket_date
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 1
+PREHOOK: query: create external table tbl_target_truncate_str (a int, ccy string) partitioned by spec (truncate(2, ccy)) stored by iceberg stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@tbl_target_truncate_str
+POSTHOOK: query: create external table tbl_target_truncate_str (a int, ccy string) partitioned by spec (truncate(2, ccy)) stored by iceberg stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@tbl_target_truncate_str
+PREHOOK: query: explain insert into table tbl_target_truncate_str select a, b from tbl_src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_src
+PREHOOK: Output: default@tbl_target_truncate_str
+POSTHOOK: query: explain insert into table tbl_target_truncate_str select a, b from tbl_src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_src
+POSTHOOK: Output: default@tbl_target_truncate_str
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 2 <- Map 1 (SIMPLE_EDGE)
+Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+
+Stage-3
+  Stats Work{}
+    Stage-0
+      Move Operator
+        table:{"name:":"default.tbl_target_truncate_str"}
+        Stage-2
+          Dependency Collection{}
+            Stage-1
+              Reducer 2 vectorized
+              File Output Operator [FS_18]
+                table:{"name:":"default.tbl_target_truncate_str"}
+                Select Operator [SEL_17]
+                  Output:["_col0","_col1","iceberg_truncate(_col1, 2)"]
+                <-Map 1 [SIMPLE_EDGE] vectorized
+                  PARTITION_ONLY_SHUFFLE [RS_13]
+                    PartitionCols:iceberg_truncate(_col1, 2)
+                    Select Operator [SEL_12] (rows=22 width=87)
+                      Output:["_col0","_col1"]
+                      TableScan [TS_0] (rows=22 width=87)
+                        default@tbl_src,tbl_src,Tbl:COMPLETE,Col:COMPLETE,Output:["a","b"]
+              Reducer 3 vectorized
+              File Output Operator [FS_21]
+                Select Operator [SEL_20] (rows=1 width=530)
+                  Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
+                  Group By Operator [GBY_19] (rows=1 width=332)
+                    Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","max(VALUE._col5)","avg(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)"]
+                  <-Map 1 [CUSTOM_SIMPLE_EDGE] vectorized
+                    PARTITION_ONLY_SHUFFLE [RS_16]
+                      Group By Operator [GBY_15] (rows=1 width=400)
+                        Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["min(a)","max(a)","count(1)","count(a)","compute_bit_vector_hll(a)","max(length(ccy))","avg(COALESCE(length(ccy),0))","count(ccy)","compute_bit_vector_hll(ccy)"]
+                        Select Operator [SEL_14] (rows=22 width=87)
+                          Output:["a","ccy"]
+                           Please refer to the previous Select Operator [SEL_12]
+
+PREHOOK: query: insert into table tbl_target_truncate_str select a, b from tbl_src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_src
+PREHOOK: Output: default@tbl_target_truncate_str
+POSTHOOK: query: insert into table tbl_target_truncate_str select a, b from tbl_src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_src
+POSTHOOK: Output: default@tbl_target_truncate_str
+PREHOOK: query: select * from tbl_target_truncate_str order by a, ccy
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_target_truncate_str
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from tbl_target_truncate_str order by a, ccy
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_target_truncate_str
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+1	EUR
+2	EUR
+3	USD
+4	EUR
+5	HUF
+6	USD
+7	USD
+8	PLN
+9	PLN
+10	CZK
+10	EUR
+12	NULL
+20	EUR
+30	USD
+40	EUR
+50	HUF
+60	USD
+70	USD
+80	PLN
+90	PLN
+100	CZK
+110	NULL
+PREHOOK: query: create external table tbl_target_truncate_int (id int, ccy string) partitioned by spec (truncate(2, id)) stored by iceberg stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@tbl_target_truncate_int
+POSTHOOK: query: create external table tbl_target_truncate_int (id int, ccy string) partitioned by spec (truncate(2, id)) stored by iceberg stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@tbl_target_truncate_int
+PREHOOK: query: explain insert into table tbl_target_truncate_int select a, b from tbl_src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_src
+PREHOOK: Output: default@tbl_target_truncate_int
+POSTHOOK: query: explain insert into table tbl_target_truncate_int select a, b from tbl_src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_src
+POSTHOOK: Output: default@tbl_target_truncate_int
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 2 <- Map 1 (SIMPLE_EDGE)
+Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+
+Stage-3
+  Stats Work{}
+    Stage-0
+      Move Operator
+        table:{"name:":"default.tbl_target_truncate_int"}
+        Stage-2
+          Dependency Collection{}
+            Stage-1
+              Reducer 2 vectorized
+              File Output Operator [FS_18]
+                table:{"name:":"default.tbl_target_truncate_int"}
+                Select Operator [SEL_17]
+                  Output:["_col0","_col1","iceberg_truncate(_col0, 2)"]
+                <-Map 1 [SIMPLE_EDGE] vectorized
+                  PARTITION_ONLY_SHUFFLE [RS_13]
+                    PartitionCols:iceberg_truncate(_col0, 2)
+                    Select Operator [SEL_12] (rows=22 width=87)
+                      Output:["_col0","_col1"]
+                      TableScan [TS_0] (rows=22 width=87)
+                        default@tbl_src,tbl_src,Tbl:COMPLETE,Col:COMPLETE,Output:["a","b"]
+              Reducer 3 vectorized
+              File Output Operator [FS_21]
+                Select Operator [SEL_20] (rows=1 width=530)
+                  Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
+                  Group By Operator [GBY_19] (rows=1 width=332)
+                    Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","max(VALUE._col5)","avg(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)"]
+                  <-Map 1 [CUSTOM_SIMPLE_EDGE] vectorized
+                    PARTITION_ONLY_SHUFFLE [RS_16]
+                      Group By Operator [GBY_15] (rows=1 width=400)
+                        Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["min(id)","max(id)","count(1)","count(id)","compute_bit_vector_hll(id)","max(length(ccy))","avg(COALESCE(length(ccy),0))","count(ccy)","compute_bit_vector_hll(ccy)"]
+                        Select Operator [SEL_14] (rows=22 width=87)
+                          Output:["id","ccy"]
+                           Please refer to the previous Select Operator [SEL_12]
+
+PREHOOK: query: insert into table tbl_target_truncate_int select a, b from tbl_src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_src
+PREHOOK: Output: default@tbl_target_truncate_int
+POSTHOOK: query: insert into table tbl_target_truncate_int select a, b from tbl_src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_src
+POSTHOOK: Output: default@tbl_target_truncate_int
+PREHOOK: query: select * from tbl_target_truncate_int order by id, ccy
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_target_truncate_int
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from tbl_target_truncate_int order by id, ccy
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_target_truncate_int
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+1	EUR
+2	EUR
+3	USD
+4	EUR
+5	HUF
+6	USD
+7	USD
+8	PLN
+9	PLN
+10	CZK
+10	EUR
+12	NULL
+20	EUR
+30	USD
+40	EUR
+50	HUF
+60	USD
+70	USD
+80	PLN
+90	PLN
+100	CZK
+110	NULL
+PREHOOK: query: create external table tbl_target_truncate_bigint (a int, ccy bigint) partitioned by spec (truncate(2, ccy)) stored by iceberg stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@tbl_target_truncate_bigint
+POSTHOOK: query: create external table tbl_target_truncate_bigint (a int, ccy bigint) partitioned by spec (truncate(2, ccy)) stored by iceberg stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@tbl_target_truncate_bigint
+PREHOOK: query: explain insert into table tbl_target_truncate_bigint select a, c from tbl_src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_src
+PREHOOK: Output: default@tbl_target_truncate_bigint
+POSTHOOK: query: explain insert into table tbl_target_truncate_bigint select a, c from tbl_src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_src
+POSTHOOK: Output: default@tbl_target_truncate_bigint
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 2 <- Map 1 (SIMPLE_EDGE)
+Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+
+Stage-3
+  Stats Work{}
+    Stage-0
+      Move Operator
+        table:{"name:":"default.tbl_target_truncate_bigint"}
+        Stage-2
+          Dependency Collection{}
+            Stage-1
+              Reducer 2 vectorized
+              File Output Operator [FS_18]
+                table:{"name:":"default.tbl_target_truncate_bigint"}
+                Select Operator [SEL_17]
+                  Output:["_col0","_col1","iceberg_truncate(_col1, 2)"]
+                <-Map 1 [SIMPLE_EDGE] vectorized
+                  PARTITION_ONLY_SHUFFLE [RS_13]
+                    PartitionCols:iceberg_truncate(_col1, 2)
+                    Select Operator [SEL_12] (rows=22 width=11)
+                      Output:["_col0","_col1"]
+                      TableScan [TS_0] (rows=22 width=11)
+                        default@tbl_src,tbl_src,Tbl:COMPLETE,Col:COMPLETE,Output:["a","c"]
+              Reducer 3 vectorized
+              File Output Operator [FS_21]
+                Select Operator [SEL_20] (rows=1 width=528)
+                  Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
+                  Group By Operator [GBY_19] (rows=1 width=336)
+                    Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)"]
+                  <-Map 1 [CUSTOM_SIMPLE_EDGE] vectorized
+                    PARTITION_ONLY_SHUFFLE [RS_16]
+                      Group By Operator [GBY_15] (rows=1 width=336)
+                        Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["min(a)","max(a)","count(1)","count(a)","compute_bit_vector_hll(a)","min(ccy)","max(ccy)","count(ccy)","compute_bit_vector_hll(ccy)"]
+                        Select Operator [SEL_14] (rows=22 width=11)
+                          Output:["a","ccy"]
+                           Please refer to the previous Select Operator [SEL_12]
+
+PREHOOK: query: insert into table tbl_target_truncate_bigint select a, c from tbl_src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_src
+PREHOOK: Output: default@tbl_target_truncate_bigint
+POSTHOOK: query: insert into table tbl_target_truncate_bigint select a, c from tbl_src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_src
+POSTHOOK: Output: default@tbl_target_truncate_bigint
+PREHOOK: query: select * from tbl_target_truncate_bigint order by a, ccy
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_target_truncate_bigint
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from tbl_target_truncate_bigint order by a, ccy
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_target_truncate_bigint
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+1	10
+2	10
+3	11
+4	12
+5	30
+6	10
+7	100
+8	20
+9	11
+10	5
+10	12
+12	NULL
+20	11
+30	100
+40	10
+50	30
+60	12
+70	20
+80	100
+90	18
+100	12
+110	NULL
+PREHOOK: query: create external table tbl_target_truncate_decimal (a int, b string, ccy decimal(10,6)) partitioned by spec (truncate(2, b), truncate(3, ccy)) stored by iceberg stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@tbl_target_truncate_decimal
+POSTHOOK: query: create external table tbl_target_truncate_decimal (a int, b string, ccy decimal(10,6)) partitioned by spec (truncate(2, b), truncate(3, ccy)) stored by iceberg stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@tbl_target_truncate_decimal
+PREHOOK: query: explain insert into table tbl_target_truncate_decimal select a, b, 1.567894 from tbl_src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_src
+PREHOOK: Output: default@tbl_target_truncate_decimal
+POSTHOOK: query: explain insert into table tbl_target_truncate_decimal select a, b, 1.567894 from tbl_src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_src
+POSTHOOK: Output: default@tbl_target_truncate_decimal
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 2 <- Map 1 (SIMPLE_EDGE)
+Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+
+Stage-3
+  Stats Work{}
+    Stage-0
+      Move Operator
+        table:{"name:":"default.tbl_target_truncate_decimal"}
+        Stage-2
+          Dependency Collection{}
+            Stage-1
+              Reducer 2 vectorized
+              File Output Operator [FS_19]
+                table:{"name:":"default.tbl_target_truncate_decimal"}
+                Select Operator [SEL_18]
+                  Output:["_col0","_col1","_col2","iceberg_truncate(_col1, 2)","iceberg_truncate(_col2, 3)"]
+                <-Map 1 [SIMPLE_EDGE] vectorized
+                  PARTITION_ONLY_SHUFFLE [RS_14]
+                    PartitionCols:iceberg_truncate(_col1, 2), iceberg_truncate(_col2, 3)
+                    Select Operator [SEL_13] (rows=22 width=199)
+                      Output:["_col0","_col1","_col2"]
+                      TableScan [TS_0] (rows=22 width=87)
+                        default@tbl_src,tbl_src,Tbl:COMPLETE,Col:COMPLETE,Output:["a","b"]
+              Reducer 3 vectorized
+              File Output Operator [FS_22]
+                Select Operator [SEL_21] (rows=1 width=1005)
+                  Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"]
+                  Group By Operator [GBY_20] (rows=1 width=708)
+                    Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","max(VALUE._col5)","avg(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"]
+                  <-Map 1 [CUSTOM_SIMPLE_EDGE] vectorized
+                    PARTITION_ONLY_SHUFFLE [RS_17]
+                      Group By Operator [GBY_16] (rows=1 width=776)
+                        Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["min(a)","max(a)","count(1)","count(a)","compute_bit_vector_hll(a)","max(length(b))","avg(COALESCE(length(b),0))","count(b)","compute_bit_vector_hll(b)","min(ccy)","max(ccy)","count(ccy)","compute_bit_vector_hll(ccy)"]
+                        Select Operator [SEL_15] (rows=22 width=199)
+                          Output:["a","b","ccy"]
+                           Please refer to the previous Select Operator [SEL_13]
+
+PREHOOK: query: insert into table tbl_target_truncate_decimal select a, b, 1.567894 from tbl_src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_src
+PREHOOK: Output: default@tbl_target_truncate_decimal
+POSTHOOK: query: insert into table tbl_target_truncate_decimal select a, b, 1.567894 from tbl_src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_src
+POSTHOOK: Output: default@tbl_target_truncate_decimal
+PREHOOK: query: select * from tbl_target_truncate_decimal order by a, b
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_target_truncate_decimal
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from tbl_target_truncate_decimal order by a, b
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_target_truncate_decimal
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+1	EUR	1.567894
+2	EUR	1.567894
+3	USD	1.567894
+4	EUR	1.567894
+5	HUF	1.567894
+6	USD	1.567894
+7	USD	1.567894
+8	PLN	1.567894
+9	PLN	1.567894
+10	CZK	1.567894
+10	EUR	1.567894
+12	NULL	1.567894
+20	EUR	1.567894
+30	USD	1.567894
+40	EUR	1.567894
+50	HUF	1.567894
+60	USD	1.567894
+70	USD	1.567894
+80	PLN	1.567894
+90	PLN	1.567894
+100	CZK	1.567894
+110	NULL	1.567894
+PREHOOK: query: create external table tbl_year_date (id string, date_time_date date, year_partition int)
+    partitioned by spec (year_partition, year(date_time_date))
+stored by iceberg stored as parquet
+tblproperties ('parquet.compression'='snappy','format-version'='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@tbl_year_date
+POSTHOOK: query: create external table tbl_year_date (id string, date_time_date date, year_partition int)
+    partitioned by spec (year_partition, year(date_time_date))
+stored by iceberg stored as parquet
+tblproperties ('parquet.compression'='snappy','format-version'='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@tbl_year_date
+PREHOOK: query: explain insert into tbl_year_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@tbl_year_date
+POSTHOOK: query: explain insert into tbl_year_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@tbl_year_date
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 2 <- Map 1 (SIMPLE_EDGE)
+Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+
+Stage-3
+  Stats Work{}
+    Stage-0
+      Move Operator
+        table:{"name:":"default.tbl_year_date"}
+        Stage-2
+          Dependency Collection{}
+            Stage-1
+              Reducer 2 vectorized
+              File Output Operator [FS_16]
+                table:{"name:":"default.tbl_year_date"}
+                Select Operator [SEL_15]
+                  Output:["_col0","_col1","_col2","_col2","iceberg_year(_col1)"]
+                <-Map 1 [SIMPLE_EDGE]
+                  PARTITION_ONLY_SHUFFLE [RS_13]
+                    PartitionCols:_col2, iceberg_year(_col1)
+                    Select Operator [SEL_3] (rows=1 width=240)
+                      Output:["_col0","_col1","_col2"]
+                      UDTF Operator [UDTF_2] (rows=1 width=64)
+                        function name:inline
+                        Select Operator [SEL_1] (rows=1 width=64)
+                          Output:["_col0"]
+                          TableScan [TS_0] (rows=1 width=10)
+                            _dummy_database@_dummy_table,_dummy_table,Tbl:COMPLETE,Col:COMPLETE
+              Reducer 3 vectorized
+              File Output Operator [FS_19]
+                Select Operator [SEL_18] (rows=1 width=890)
+                  Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"]
+                  Group By Operator [GBY_17] (rows=1 width=596)
+                    Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"]
+                  <-Map 1 [CUSTOM_SIMPLE_EDGE]
+                    PARTITION_ONLY_SHUFFLE [RS_9]
+                      Group By Operator [GBY_8] (rows=1 width=664)
+                        Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(length(id))","avg(COALESCE(length(id),0))","count(1)","count(id)","compute_bit_vector_hll(id)","min(date_time_date)","max(date_time_date)","count(date_time_date)","compute_bit_vector_hll(date_time_date)","min(year_partition)","max(year_partition)","count(year_partition)","compute_bit_vector_hll(year_partition)"]
+                        Select Operator [SEL_7] (rows=1 width=240)
+                          Output:["id","date_time_date","year_partition"]
+                           Please refer to the previous Select Operator [SEL_3]
+
+PREHOOK: query: insert into tbl_year_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@tbl_year_date
+POSTHOOK: query: insert into tbl_year_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@tbl_year_date
+PREHOOK: query: select * from tbl_year_date order by id, date_time_date
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_year_date
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from tbl_year_date order by id, date_time_date
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_year_date
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+40568	2018-02-12	2018
+40568	2018-07-03	2018
+88669	2018-05-27	2018
+PREHOOK: query: create external table tbl_year_timestamp (id string, date_time_timestamp timestamp, year_partition int)
+    partitioned by spec (year_partition, year(date_time_timestamp))
+stored by iceberg stored as parquet
+tblproperties ('parquet.compression'='snappy','format-version'='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@tbl_year_timestamp
+POSTHOOK: query: create external table tbl_year_timestamp (id string, date_time_timestamp timestamp, year_partition int)
+    partitioned by spec (year_partition, year(date_time_timestamp))
+stored by iceberg stored as parquet
+tblproperties ('parquet.compression'='snappy','format-version'='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@tbl_year_timestamp
+PREHOOK: query: explain insert into tbl_year_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@tbl_year_timestamp
+POSTHOOK: query: explain insert into tbl_year_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@tbl_year_timestamp
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 2 <- Map 1 (SIMPLE_EDGE)
+Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+
+Stage-3
+  Stats Work{}
+    Stage-0
+      Move Operator
+        table:{"name:":"default.tbl_year_timestamp"}
+        Stage-2
+          Dependency Collection{}
+            Stage-1
+              Reducer 2 vectorized
+              File Output Operator [FS_16]
+                table:{"name:":"default.tbl_year_timestamp"}
+                Select Operator [SEL_15]
+                  Output:["_col0","_col1","_col2","_col2","iceberg_year(_col1)"]
+                <-Map 1 [SIMPLE_EDGE]
+                  PARTITION_ONLY_SHUFFLE [RS_13]
+                    PartitionCols:_col2, iceberg_year(_col1)
+                    Select Operator [SEL_3] (rows=1 width=224)
+                      Output:["_col0","_col1","_col2"]
+                      UDTF Operator [UDTF_2] (rows=1 width=64)
+                        function name:inline
+                        Select Operator [SEL_1] (rows=1 width=64)
+                          Output:["_col0"]
+                          TableScan [TS_0] (rows=1 width=10)
+                            _dummy_database@_dummy_table,_dummy_table,Tbl:COMPLETE,Col:COMPLETE
+              Reducer 3 vectorized
+              File Output Operator [FS_19]
+                Select Operator [SEL_18] (rows=1 width=863)
+                  Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"]
+                  Group By Operator [GBY_17] (rows=1 width=564)
+                    Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"]
+                  <-Map 1 [CUSTOM_SIMPLE_EDGE]
+                    PARTITION_ONLY_SHUFFLE [RS_9]
+                      Group By Operator [GBY_8] (rows=1 width=632)
+                        Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(length(id))","avg(COALESCE(length(id),0))","count(1)","count(id)","compute_bit_vector_hll(id)","min(date_time_timestamp)","max(date_time_timestamp)","count(date_time_timestamp)","compute_bit_vector_hll(date_time_timestamp)","min(year_partition)","max(year_partition)","count(year_partition)","compute_bit_vector_hll(year_partition)"]
+                        Select Operator [SEL_7] (rows=1 width=224)
+                          Output:["id","date_time_timestamp","year_partition"]
+                           Please refer to the previous Select Operator [SEL_3]
+
+PREHOOK: query: insert into tbl_year_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@tbl_year_timestamp
+POSTHOOK: query: insert into tbl_year_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@tbl_year_timestamp
+PREHOOK: query: select * from tbl_year_timestamp order by id, date_time_timestamp
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_year_timestamp
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from tbl_year_timestamp order by id, date_time_timestamp
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_year_timestamp
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+40568	2018-02-12 12:45:56	2018
+40568	2018-07-03 06:07:56	2018
+88669	2018-05-27 11:12:00	2018
+PREHOOK: query: create external table tbl_month_date (id string, date_time_date date, year_partition int)
+    partitioned by spec (year_partition, month(date_time_date))
+stored by iceberg stored as parquet
+tblproperties ('parquet.compression'='snappy','format-version'='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@tbl_month_date
+POSTHOOK: query: create external table tbl_month_date (id string, date_time_date date, year_partition int)
+    partitioned by spec (year_partition, month(date_time_date))
+stored by iceberg stored as parquet
+tblproperties ('parquet.compression'='snappy','format-version'='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@tbl_month_date
+PREHOOK: query: explain insert into tbl_month_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@tbl_month_date
+POSTHOOK: query: explain insert into tbl_month_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@tbl_month_date
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 2 <- Map 1 (SIMPLE_EDGE)
+Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+
+Stage-3
+  Stats Work{}
+    Stage-0
+      Move Operator
+        table:{"name:":"default.tbl_month_date"}
+        Stage-2
+          Dependency Collection{}
+            Stage-1
+              Reducer 2 vectorized
+              File Output Operator [FS_16]
+                table:{"name:":"default.tbl_month_date"}
+                Select Operator [SEL_15]
+                  Output:["_col0","_col1","_col2","_col2","iceberg_month(_col1)"]
+                <-Map 1 [SIMPLE_EDGE]
+                  PARTITION_ONLY_SHUFFLE [RS_13]
+                    PartitionCols:_col2, iceberg_month(_col1)
+                    Select Operator [SEL_3] (rows=1 width=240)
+                      Output:["_col0","_col1","_col2"]
+                      UDTF Operator [UDTF_2] (rows=1 width=64)
+                        function name:inline
+                        Select Operator [SEL_1] (rows=1 width=64)
+                          Output:["_col0"]
+                          TableScan [TS_0] (rows=1 width=10)
+                            _dummy_database@_dummy_table,_dummy_table,Tbl:COMPLETE,Col:COMPLETE
+              Reducer 3 vectorized
+              File Output Operator [FS_19]
+                Select Operator [SEL_18] (rows=1 width=890)
+                  Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"]
+                  Group By Operator [GBY_17] (rows=1 width=596)
+                    Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"]
+                  <-Map 1 [CUSTOM_SIMPLE_EDGE]
+                    PARTITION_ONLY_SHUFFLE [RS_9]
+                      Group By Operator [GBY_8] (rows=1 width=664)
+                        Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(length(id))","avg(COALESCE(length(id),0))","count(1)","count(id)","compute_bit_vector_hll(id)","min(date_time_date)","max(date_time_date)","count(date_time_date)","compute_bit_vector_hll(date_time_date)","min(year_partition)","max(year_partition)","count(year_partition)","compute_bit_vector_hll(year_partition)"]
+                        Select Operator [SEL_7] (rows=1 width=240)
+                          Output:["id","date_time_date","year_partition"]
+                           Please refer to the previous Select Operator [SEL_3]
+
+PREHOOK: query: insert into tbl_month_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@tbl_month_date
+POSTHOOK: query: insert into tbl_month_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@tbl_month_date
+PREHOOK: query: select * from tbl_month_date order by id, date_time_date
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_month_date
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from tbl_month_date order by id, date_time_date
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_month_date
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+40568	2018-02-12	2018
+40568	2018-07-03	2018
+88669	2018-05-27	2018
+PREHOOK: query: create external table tbl_month_timestamp (id string, date_time_timestamp timestamp, year_partition int)
+    partitioned by spec (year_partition, month(date_time_timestamp))
+stored by iceberg stored as parquet
+tblproperties ('parquet.compression'='snappy','format-version'='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@tbl_month_timestamp
+POSTHOOK: query: create external table tbl_month_timestamp (id string, date_time_timestamp timestamp, year_partition int)
+    partitioned by spec (year_partition, month(date_time_timestamp))
+stored by iceberg stored as parquet
+tblproperties ('parquet.compression'='snappy','format-version'='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@tbl_month_timestamp
+PREHOOK: query: explain insert into tbl_month_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@tbl_month_timestamp
+POSTHOOK: query: explain insert into tbl_month_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@tbl_month_timestamp
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 2 <- Map 1 (SIMPLE_EDGE)
+Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+
+Stage-3
+  Stats Work{}
+    Stage-0
+      Move Operator
+        table:{"name:":"default.tbl_month_timestamp"}
+        Stage-2
+          Dependency Collection{}
+            Stage-1
+              Reducer 2 vectorized
+              File Output Operator [FS_16]
+                table:{"name:":"default.tbl_month_timestamp"}
+                Select Operator [SEL_15]
+                  Output:["_col0","_col1","_col2","_col2","iceberg_month(_col1)"]
+                <-Map 1 [SIMPLE_EDGE]
+                  PARTITION_ONLY_SHUFFLE [RS_13]
+                    PartitionCols:_col2, iceberg_month(_col1)
+                    Select Operator [SEL_3] (rows=1 width=224)
+                      Output:["_col0","_col1","_col2"]
+                      UDTF Operator [UDTF_2] (rows=1 width=64)
+                        function name:inline
+                        Select Operator [SEL_1] (rows=1 width=64)
+                          Output:["_col0"]
+                          TableScan [TS_0] (rows=1 width=10)
+                            _dummy_database@_dummy_table,_dummy_table,Tbl:COMPLETE,Col:COMPLETE
+              Reducer 3 vectorized
+              File Output Operator [FS_19]
+                Select Operator [SEL_18] (rows=1 width=863)
+                  Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"]
+                  Group By Operator [GBY_17] (rows=1 width=564)
+                    Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"]
+                  <-Map 1 [CUSTOM_SIMPLE_EDGE]
+                    PARTITION_ONLY_SHUFFLE [RS_9]
+                      Group By Operator [GBY_8] (rows=1 width=632)
+                        Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(length(id))","avg(COALESCE(length(id),0))","count(1)","count(id)","compute_bit_vector_hll(id)","min(date_time_timestamp)","max(date_time_timestamp)","count(date_time_timestamp)","compute_bit_vector_hll(date_time_timestamp)","min(year_partition)","max(year_partition)","count(year_partition)","compute_bit_vector_hll(year_partition)"]
+                        Select Operator [SEL_7] (rows=1 width=224)
+                          Output:["id","date_time_timestamp","year_partition"]
+                           Please refer to the previous Select Operator [SEL_3]
+
+PREHOOK: query: insert into tbl_month_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@tbl_month_timestamp
+POSTHOOK: query: insert into tbl_month_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@tbl_month_timestamp
+PREHOOK: query: select * from tbl_month_timestamp order by id, date_time_timestamp
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_month_timestamp
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from tbl_month_timestamp order by id, date_time_timestamp
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_month_timestamp
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+40568	2018-02-12 12:45:56	2018
+40568	2018-07-03 06:07:56	2018
+88669	2018-05-27 11:12:00	2018
+PREHOOK: query: create external table tbl_day_date (id string, date_time_date date, year_partition int)
+    partitioned by spec (year_partition, day(date_time_date))
+stored by iceberg stored as parquet
+tblproperties ('parquet.compression'='snappy','format-version'='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@tbl_day_date
+POSTHOOK: query: create external table tbl_day_date (id string, date_time_date date, year_partition int)
+    partitioned by spec (year_partition, day(date_time_date))
+stored by iceberg stored as parquet
+tblproperties ('parquet.compression'='snappy','format-version'='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@tbl_day_date
+PREHOOK: query: explain insert into tbl_day_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@tbl_day_date
+POSTHOOK: query: explain insert into tbl_day_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@tbl_day_date
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 2 <- Map 1 (SIMPLE_EDGE)
+Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+
+Stage-3
+  Stats Work{}
+    Stage-0
+      Move Operator
+        table:{"name:":"default.tbl_day_date"}
+        Stage-2
+          Dependency Collection{}
+            Stage-1
+              Reducer 2 vectorized
+              File Output Operator [FS_16]
+                table:{"name:":"default.tbl_day_date"}
+                Select Operator [SEL_15]
+                  Output:["_col0","_col1","_col2","_col2","iceberg_day(_col1)"]
+                <-Map 1 [SIMPLE_EDGE]
+                  PARTITION_ONLY_SHUFFLE [RS_13]
+                    PartitionCols:_col2, iceberg_day(_col1)
+                    Select Operator [SEL_3] (rows=1 width=240)
+                      Output:["_col0","_col1","_col2"]
+                      UDTF Operator [UDTF_2] (rows=1 width=64)
+                        function name:inline
+                        Select Operator [SEL_1] (rows=1 width=64)
+                          Output:["_col0"]
+                          TableScan [TS_0] (rows=1 width=10)
+                            _dummy_database@_dummy_table,_dummy_table,Tbl:COMPLETE,Col:COMPLETE
+              Reducer 3 vectorized
+              File Output Operator [FS_19]
+                Select Operator [SEL_18] (rows=1 width=890)
+                  Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"]
+                  Group By Operator [GBY_17] (rows=1 width=596)
+                    Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"]
+                  <-Map 1 [CUSTOM_SIMPLE_EDGE]
+                    PARTITION_ONLY_SHUFFLE [RS_9]
+                      Group By Operator [GBY_8] (rows=1 width=664)
+                        Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(length(id))","avg(COALESCE(length(id),0))","count(1)","count(id)","compute_bit_vector_hll(id)","min(date_time_date)","max(date_time_date)","count(date_time_date)","compute_bit_vector_hll(date_time_date)","min(year_partition)","max(year_partition)","count(year_partition)","compute_bit_vector_hll(year_partition)"]
+                        Select Operator [SEL_7] (rows=1 width=240)
+                          Output:["id","date_time_date","year_partition"]
+                           Please refer to the previous Select Operator [SEL_3]
+
+PREHOOK: query: insert into tbl_day_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@tbl_day_date
+POSTHOOK: query: insert into tbl_day_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@tbl_day_date
+PREHOOK: query: select * from tbl_day_date order by id, date_time_date
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_day_date
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from tbl_day_date order by id, date_time_date
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_day_date
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+40568	2018-02-12	2018
+40568	2018-07-03	2018
+88669	2018-05-27	2018
+PREHOOK: query: create external table tbl_day_timestamp (id string, date_time_timestamp timestamp, year_partition int)
+    partitioned by spec (year_partition, day(date_time_timestamp))
+stored by iceberg stored as parquet
+tblproperties ('parquet.compression'='snappy','format-version'='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@tbl_day_timestamp
+POSTHOOK: query: create external table tbl_day_timestamp (id string, date_time_timestamp timestamp, year_partition int)
+    partitioned by spec (year_partition, day(date_time_timestamp))
+stored by iceberg stored as parquet
+tblproperties ('parquet.compression'='snappy','format-version'='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@tbl_day_timestamp
+PREHOOK: query: explain insert into tbl_day_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@tbl_day_timestamp
+POSTHOOK: query: explain insert into tbl_day_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@tbl_day_timestamp
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 2 <- Map 1 (SIMPLE_EDGE)
+Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+
+Stage-3
+  Stats Work{}
+    Stage-0
+      Move Operator
+        table:{"name:":"default.tbl_day_timestamp"}
+        Stage-2
+          Dependency Collection{}
+            Stage-1
+              Reducer 2 vectorized
+              File Output Operator [FS_16]
+                table:{"name:":"default.tbl_day_timestamp"}
+                Select Operator [SEL_15]
+                  Output:["_col0","_col1","_col2","_col2","iceberg_day(_col1)"]
+                <-Map 1 [SIMPLE_EDGE]
+                  PARTITION_ONLY_SHUFFLE [RS_13]
+                    PartitionCols:_col2, iceberg_day(_col1)
+                    Select Operator [SEL_3] (rows=1 width=224)
+                      Output:["_col0","_col1","_col2"]
+                      UDTF Operator [UDTF_2] (rows=1 width=64)
+                        function name:inline
+                        Select Operator [SEL_1] (rows=1 width=64)
+                          Output:["_col0"]
+                          TableScan [TS_0] (rows=1 width=10)
+                            _dummy_database@_dummy_table,_dummy_table,Tbl:COMPLETE,Col:COMPLETE
+              Reducer 3 vectorized
+              File Output Operator [FS_19]
+                Select Operator [SEL_18] (rows=1 width=863)
+                  Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"]
+                  Group By Operator [GBY_17] (rows=1 width=564)
+                    Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"]
+                  <-Map 1 [CUSTOM_SIMPLE_EDGE]
+                    PARTITION_ONLY_SHUFFLE [RS_9]
+                      Group By Operator [GBY_8] (rows=1 width=632)
+                        Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(length(id))","avg(COALESCE(length(id),0))","count(1)","count(id)","compute_bit_vector_hll(id)","min(date_time_timestamp)","max(date_time_timestamp)","count(date_time_timestamp)","compute_bit_vector_hll(date_time_timestamp)","min(year_partition)","max(year_partition)","count(year_partition)","compute_bit_vector_hll(year_partition)"]
+                        Select Operator [SEL_7] (rows=1 width=224)
+                          Output:["id","date_time_timestamp","year_partition"]
+                           Please refer to the previous Select Operator [SEL_3]
+
+PREHOOK: query: insert into tbl_day_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@tbl_day_timestamp
+POSTHOOK: query: insert into tbl_day_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@tbl_day_timestamp
+PREHOOK: query: select * from tbl_day_timestamp order by id, date_time_timestamp
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_day_timestamp
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from tbl_day_timestamp order by id, date_time_timestamp
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_day_timestamp
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+40568	2018-02-12 12:45:56	2018
+40568	2018-07-03 06:07:56	2018
+88669	2018-05-27 11:12:00	2018
+PREHOOK: query: create external table tbl_hour_timestamp (id string, date_time_timestamp timestamp, year_partition int)
+    partitioned by spec (year_partition, hour(date_time_timestamp))
+stored by iceberg stored as parquet
+tblproperties ('parquet.compression'='snappy','format-version'='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@tbl_hour_timestamp
+POSTHOOK: query: create external table tbl_hour_timestamp (id string, date_time_timestamp timestamp, year_partition int)
+    partitioned by spec (year_partition, hour(date_time_timestamp))
+stored by iceberg stored as parquet
+tblproperties ('parquet.compression'='snappy','format-version'='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@tbl_hour_timestamp
+PREHOOK: query: explain insert into tbl_hour_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@tbl_hour_timestamp
+POSTHOOK: query: explain insert into tbl_hour_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@tbl_hour_timestamp
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 2 <- Map 1 (SIMPLE_EDGE)
+Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+
+Stage-3
+  Stats Work{}
+    Stage-0
+      Move Operator
+        table:{"name:":"default.tbl_hour_timestamp"}
+        Stage-2
+          Dependency Collection{}
+            Stage-1
+              Reducer 2 vectorized
+              File Output Operator [FS_16]
+                table:{"name:":"default.tbl_hour_timestamp"}
+                Select Operator [SEL_15]
+                  Output:["_col0","_col1","_col2","_col2","iceberg_hour(_col1)"]
+                <-Map 1 [SIMPLE_EDGE]
+                  PARTITION_ONLY_SHUFFLE [RS_13]
+                    PartitionCols:_col2, iceberg_hour(_col1)
+                    Select Operator [SEL_3] (rows=1 width=224)
+                      Output:["_col0","_col1","_col2"]
+                      UDTF Operator [UDTF_2] (rows=1 width=64)
+                        function name:inline
+                        Select Operator [SEL_1] (rows=1 width=64)
+                          Output:["_col0"]
+                          TableScan [TS_0] (rows=1 width=10)
+                            _dummy_database@_dummy_table,_dummy_table,Tbl:COMPLETE,Col:COMPLETE
+              Reducer 3 vectorized
+              File Output Operator [FS_19]
+                Select Operator [SEL_18] (rows=1 width=863)
+                  Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"]
+                  Group By Operator [GBY_17] (rows=1 width=564)
+                    Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"]
+                  <-Map 1 [CUSTOM_SIMPLE_EDGE]
+                    PARTITION_ONLY_SHUFFLE [RS_9]
+                      Group By Operator [GBY_8] (rows=1 width=632)
+                        Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(length(id))","avg(COALESCE(length(id),0))","count(1)","count(id)","compute_bit_vector_hll(id)","min(date_time_timestamp)","max(date_time_timestamp)","count(date_time_timestamp)","compute_bit_vector_hll(date_time_timestamp)","min(year_partition)","max(year_partition)","count(year_partition)","compute_bit_vector_hll(year_partition)"]
+                        Select Operator [SEL_7] (rows=1 width=224)
+                          Output:["id","date_time_timestamp","year_partition"]
+                           Please refer to the previous Select Operator [SEL_3]
+
+PREHOOK: query: insert into tbl_hour_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@tbl_hour_timestamp
+POSTHOOK: query: insert into tbl_hour_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@tbl_hour_timestamp
+PREHOOK: query: select * from tbl_hour_timestamp order by id, date_time_timestamp
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_hour_timestamp
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from tbl_hour_timestamp order by id, date_time_timestamp
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_hour_timestamp
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+40568	2018-02-12 12:45:56	2018
+40568	2018-07-03 06:07:56	2018
+88669	2018-05-27 11:12:00	2018
diff --git a/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_partitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_partitioned.q.out
index 1227db6101d..308d2a8d62a 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_partitioned.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_partitioned.q.out
@@ -228,18 +228,18 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                   Statistics: Num rows: 1 Data size: 302 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
-                    key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                    key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                     null sort order: aa
                     sort order: ++
-                    Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                    Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                     Statistics: Num rows: 7 Data size: 2096 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int)
         Reducer 4 
             Execution mode: vectorized
             Reduce Operator Tree:
               Select Operator
-                expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY._col5 (type: string)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), _col5
+                expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY.iceberg_truncate(_col5, 3) (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), iceberg_truncate(_col5, 3)
                 File Output Operator
                   compressed: false
                   Dp Sort State: PARTITION_SORTED
@@ -322,10 +322,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                 Statistics: Num rows: 4 Data size: 1196 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
-                  key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                  key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                   null sort order: aa
                   sort order: ++
-                  Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                  Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                   Statistics: Num rows: 7 Data size: 2096 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int)
         Reducer 8 
@@ -379,10 +379,10 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                       Statistics: Num rows: 2 Data size: 598 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                        key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                         null sort order: aa
                         sort order: ++
-                        Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                        Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                         Statistics: Num rows: 7 Data size: 2096 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int)
         Union 3 
@@ -511,18 +511,18 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                     Statistics: Num rows: 9 Data size: 900 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
-                      key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                      key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                       null sort order: aa
                       sort order: ++
-                      Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                      Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                       Statistics: Num rows: 9 Data size: 900 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int)
         Reducer 3 
             Execution mode: vectorized
             Reduce Operator Tree:
               Select Operator
-                expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY._col5 (type: string)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), _col5
+                expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY.iceberg_truncate(_col5, 3) (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), iceberg_truncate(_col5, 3)
                 File Output Operator
                   compressed: false
                   Dp Sort State: PARTITION_SORTED
diff --git a/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_partitioned_orc.q.out b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_partitioned_orc.q.out
index f8cc1fa1559..f20856fd291 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_partitioned_orc.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_partitioned_orc.q.out
@@ -162,10 +162,10 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 1 Data size: 98 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: iceberg_bucket(_col0, 16) (type: int), _col1 (type: string)
+                        key expressions: iceberg_bucket(_col0, 16) (type: int), iceberg_truncate(_col1, 3) (type: string)
                         null sort order: aa
                         sort order: ++
-                        Map-reduce partition columns: iceberg_bucket(_col0, 16) (type: int), _col1 (type: string)
+                        Map-reduce partition columns: iceberg_bucket(_col0, 16) (type: int), iceberg_truncate(_col1, 3) (type: string)
                         Statistics: Num rows: 1 Data size: 98 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int)
                   Filter Operator
@@ -176,10 +176,10 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: iceberg_bucket(_col0, 16) (type: int), _col1 (type: string)
+                        key expressions: iceberg_bucket(_col0, 16) (type: int), iceberg_truncate(_col1, 3) (type: string)
                         null sort order: aa
                         sort order: ++
-                        Map-reduce partition columns: iceberg_bucket(_col0, 16) (type: int), _col1 (type: string)
+                        Map-reduce partition columns: iceberg_bucket(_col0, 16) (type: int), iceberg_truncate(_col1, 3) (type: string)
                         Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int)
                   Filter Operator
@@ -237,8 +237,8 @@ STAGE PLANS:
             Execution mode: vectorized
             Reduce Operator Tree:
               Select Operator
-                expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), KEY.iceberg_bucket(_col0, 16) (type: int), KEY._col1 (type: string)
-                outputColumnNames: _col0, _col1, _col2, iceberg_bucket(_col0, 16), _col1
+                expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), KEY.iceberg_bucket(_col0, 16) (type: int), KEY.iceberg_truncate(_col1, 3) (type: string)
+                outputColumnNames: _col0, _col1, _col2, iceberg_bucket(_col0, 16), iceberg_truncate(_col1, 3)
                 File Output Operator
                   compressed: false
                   Dp Sort State: PARTITION_SORTED
@@ -252,8 +252,8 @@ STAGE PLANS:
             Execution mode: vectorized
             Reduce Operator Tree:
               Select Operator
-                expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), KEY.iceberg_bucket(_col0, 16) (type: int), KEY._col1 (type: string)
-                outputColumnNames: _col0, _col1, _col2, iceberg_bucket(_col0, 16), _col1
+                expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), KEY.iceberg_bucket(_col0, 16) (type: int), KEY.iceberg_truncate(_col1, 3) (type: string)
+                outputColumnNames: _col0, _col1, _col2, iceberg_bucket(_col0, 16), iceberg_truncate(_col1, 3)
                 File Output Operator
                   compressed: false
                   Dp Sort State: PARTITION_SORTED
diff --git a/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_partitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_partitioned.q.out
index 985c32f342e..7a5b872fa11 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_partitioned.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_partitioned.q.out
@@ -60,10 +60,10 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                       Statistics: Num rows: 4 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                        key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                         null sort order: aa
                         sort order: ++
-                        Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                        Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                         Statistics: Num rows: 8 Data size: 2412 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int)
             Execution mode: vectorized
@@ -118,8 +118,8 @@ STAGE PLANS:
             Execution mode: vectorized
             Reduce Operator Tree:
               Select Operator
-                expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY._col5 (type: string)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), _col5
+                expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY.iceberg_truncate(_col5, 3) (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), iceberg_truncate(_col5, 3)
                 File Output Operator
                   compressed: false
                   Dp Sort State: PARTITION_SORTED
@@ -144,10 +144,10 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                   Statistics: Num rows: 2 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
-                    key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                    key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                     null sort order: aa
                     sort order: ++
-                    Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                    Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                     Statistics: Num rows: 8 Data size: 2412 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int)
         Reducer 7 
@@ -184,10 +184,10 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                       Statistics: Num rows: 2 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                        key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                         null sort order: aa
                         sort order: ++
-                        Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                        Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                         Statistics: Num rows: 8 Data size: 2412 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int)
         Reducer 8 
@@ -605,10 +605,10 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                       Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                        key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                         null sort order: aa
                         sort order: ++
-                        Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                        Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                         Statistics: Num rows: 19 Data size: 5791 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int)
         Reducer 14 
@@ -809,10 +809,10 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                     Statistics: Num rows: 9 Data size: 2781 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
-                      key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                      key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                       null sort order: aa
                       sort order: ++
-                      Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                      Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                       Statistics: Num rows: 19 Data size: 5791 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int)
         Reducer 23 
@@ -1085,18 +1085,18 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                       Statistics: Num rows: 4 Data size: 1204 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                        key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                         null sort order: aa
                         sort order: ++
-                        Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                        Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                         Statistics: Num rows: 19 Data size: 5791 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int)
         Reducer 8 
             Execution mode: vectorized
             Reduce Operator Tree:
               Select Operator
-                expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY._col5 (type: string)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), _col5
+                expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY.iceberg_truncate(_col5, 3) (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), iceberg_truncate(_col5, 3)
                 File Output Operator
                   compressed: false
                   Dp Sort State: PARTITION_SORTED
@@ -1524,18 +1524,18 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                   Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
-                    key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                    key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                     null sort order: aa
                     sort order: ++
-                    Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                    Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                     Statistics: Num rows: 3 Data size: 913 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int)
         Reducer 6 
             Execution mode: vectorized
             Reduce Operator Tree:
               Select Operator
-                expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY._col5 (type: string)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), _col5
+                expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY.iceberg_truncate(_col5, 3) (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), iceberg_truncate(_col5, 3)
                 File Output Operator
                   compressed: false
                   Dp Sort State: PARTITION_SORTED
@@ -1596,10 +1596,10 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                       Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                        key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                         null sort order: aa
                         sort order: ++
-                        Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                        Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                         Statistics: Num rows: 3 Data size: 913 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int)
         Reducer 9 
@@ -1617,10 +1617,10 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                   Statistics: Num rows: 1 Data size: 311 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
-                    key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                    key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                     null sort order: aa
                     sort order: ++
-                    Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                    Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                     Statistics: Num rows: 3 Data size: 913 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int)
         Union 5 
@@ -1946,10 +1946,10 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                       Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                        key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                         null sort order: aa
                         sort order: ++
-                        Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                        Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                         Statistics: Num rows: 3 Data size: 910 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int)
         Reducer 2 
@@ -2030,18 +2030,18 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                   Statistics: Num rows: 1 Data size: 308 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
-                    key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                    key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                     null sort order: aa
                     sort order: ++
-                    Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                    Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                     Statistics: Num rows: 3 Data size: 910 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int)
         Reducer 5 
             Execution mode: vectorized
             Reduce Operator Tree:
               Select Operator
-                expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY._col5 (type: string)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), _col5
+                expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY.iceberg_truncate(_col5, 3) (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), iceberg_truncate(_col5, 3)
                 File Output Operator
                   compressed: false
                   Dp Sort State: PARTITION_SORTED
@@ -2128,10 +2128,10 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                   Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
-                    key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                    key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                     null sort order: aa
                     sort order: ++
-                    Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string)
+                    Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string)
                     Statistics: Num rows: 3 Data size: 910 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int)
         Union 4 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
index 871980a6446..1f411971a80 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
@@ -780,7 +780,17 @@ public final class FunctionRegistry {
 
     try {
       system.registerGenericUDF("iceberg_bucket",
-          (Class<? extends GenericUDF>) Class.forName("org.apache.iceberg.mr.hive.GenericUDFIcebergBucket"));
+          (Class<? extends GenericUDF>) Class.forName("org.apache.iceberg.mr.hive.udf.GenericUDFIcebergBucket"));
+      system.registerGenericUDF("iceberg_truncate",
+          (Class<? extends GenericUDF>) Class.forName("org.apache.iceberg.mr.hive.udf.GenericUDFIcebergTruncate"));
+      system.registerGenericUDF("iceberg_year",
+          (Class<? extends GenericUDF>) Class.forName("org.apache.iceberg.mr.hive.udf.GenericUDFIcebergYear"));
+      system.registerGenericUDF("iceberg_month",
+          (Class<? extends GenericUDF>) Class.forName("org.apache.iceberg.mr.hive.udf.GenericUDFIcebergMonth"));
+      system.registerGenericUDF("iceberg_day",
+          (Class<? extends GenericUDF>) Class.forName("org.apache.iceberg.mr.hive.udf.GenericUDFIcebergDay"));
+      system.registerGenericUDF("iceberg_hour",
+          (Class<? extends GenericUDF>) Class.forName("org.apache.iceberg.mr.hive.udf.GenericUDFIcebergHour"));
     } catch (ClassNotFoundException e) {
       LOG.warn("iceberg_bucket function could not be registered");
     }