This is an automated email from the ASF dual-hosted git repository.
ayushsaxena 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 32ee445894e HIVE-28762: Iceberg: Add support for partitions with
transforms in Drop partition. (#5645). (Ayush Saxena, reviewed by Simhadri
Govindappa, Dmitriy Fingerman, Denys Kuzmenko)
32ee445894e is described below
commit 32ee445894ea33ef2703eec6cfcaee647929986e
Author: Ayush Saxena <[email protected]>
AuthorDate: Sat Feb 22 16:22:45 2025 +0530
HIVE-28762: Iceberg: Add support for partitions with transforms in Drop
partition. (#5645). (Ayush Saxena, reviewed by Simhadri Govindappa, Dmitriy
Fingerman, Denys Kuzmenko)
---
.../iceberg/mr/hive/HiveIcebergFilterFactory.java | 140 +++++++-
.../iceberg/mr/hive/HiveIcebergMetaHook.java | 44 ++-
.../iceberg_drop_partition_transform_column.q | 119 +++++++
.../iceberg_drop_partition_tranform_column.q.out | 2 +-
.../iceberg_drop_partition_transform_column.q.out | 371 +++++++++++++++++++++
.../hadoop/hive/ql/parse/IdentifiersParser.g | 18 +
.../apache/hadoop/hive/ql/parse/ParseUtils.java | 28 +-
.../apache/hadoop/hive/ql/parse/TransformSpec.java | 52 +++
8 files changed, 753 insertions(+), 21 deletions(-)
diff --git
a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
index 4db6627cbc9..9e64c785d22 100644
---
a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
+++
b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java
@@ -22,16 +22,29 @@
import java.math.BigDecimal;
import java.sql.Date;
import java.sql.Timestamp;
+import java.time.DateTimeException;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
import java.util.List;
import java.util.stream.Collectors;
+import org.apache.commons.lang3.ObjectUtils;
import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentImpl;
+import org.apache.hadoop.hive.ql.parse.TransformSpec;
import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
import org.apache.iceberg.common.DynFields;
import org.apache.iceberg.expressions.Expression;
import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.expressions.Literal;
+import org.apache.iceberg.expressions.UnboundTerm;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.transforms.Transforms;
+import org.apache.iceberg.types.Types;
import org.apache.iceberg.util.DateTimeUtil;
import org.apache.iceberg.util.NaNUtil;
@@ -97,15 +110,19 @@ private static Expression translate(ExpressionTree tree,
List<PredicateLeaf> lea
* @return Expression fully translated from Hive PredicateLeaf
*/
private static Expression translateLeaf(PredicateLeaf leaf) {
- String column = leaf.getColumnName();
+ TransformSpec transformSpec =
TransformSpec.fromStringWithColumnName(leaf.getColumnName());
+ String columnName = transformSpec.getColumnName();
+ UnboundTerm<Object> column =
+ ObjectUtils.defaultIfNull(toTerm(columnName, transformSpec),
Expressions.ref(columnName));
+
switch (leaf.getOperator()) {
case EQUALS:
- Object literal = leafToLiteral(leaf);
+ Object literal = leafToLiteral(leaf, transformSpec);
return NaNUtil.isNaN(literal) ? isNaN(column) : equal(column, literal);
case LESS_THAN:
- return lessThan(column, leafToLiteral(leaf));
+ return lessThan(column, leafToLiteral(leaf, transformSpec));
case LESS_THAN_EQUALS:
- return lessThanOrEqual(column, leafToLiteral(leaf));
+ return lessThanOrEqual(column, leafToLiteral(leaf, transformSpec));
case IN:
return in(column, leafToLiteralList(leaf));
case BETWEEN:
@@ -127,19 +144,44 @@ private static Expression translateLeaf(PredicateLeaf
leaf) {
}
}
+ public static UnboundTerm<Object> toTerm(String columnName, TransformSpec
transformSpec) {
+ if (transformSpec == null) {
+ return null;
+ }
+ switch (transformSpec.getTransformType()) {
+ case YEAR:
+ return Expressions.year(columnName);
+ case MONTH:
+ return Expressions.month(columnName);
+ case DAY:
+ return Expressions.day(columnName);
+ case HOUR:
+ return Expressions.hour(columnName);
+ case TRUNCATE:
+ return Expressions.truncate(columnName,
transformSpec.getTransformParam().get());
+ case BUCKET:
+ return Expressions.bucket(columnName,
transformSpec.getTransformParam().get());
+ case IDENTITY:
+ return null;
+ default:
+ throw new UnsupportedOperationException("Unknown transformSpec: " +
transformSpec);
+ }
+ }
+
// PredicateLeafImpl has a work-around for Kryo serialization with
java.util.Date objects where it converts values to
// Timestamp using Date#getTime. This conversion discards microseconds, so
this is a necessary to avoid it.
private static final DynFields.UnboundField<?> LITERAL_FIELD =
DynFields.builder()
.hiddenImpl(SearchArgumentImpl.PredicateLeafImpl.class, "literal")
.build();
- private static Object leafToLiteral(PredicateLeaf leaf) {
+ private static Object leafToLiteral(PredicateLeaf leaf, TransformSpec
transform) {
switch (leaf.getType()) {
case LONG:
case BOOLEAN:
- case STRING:
case FLOAT:
return leaf.getLiteral();
+ case STRING:
+ return convertLiteral(leaf.getLiteral(), transform);
case DATE:
if (leaf.getLiteral() instanceof Date) {
return daysFromDate((Date) leaf.getLiteral());
@@ -155,6 +197,92 @@ private static Object leafToLiteral(PredicateLeaf leaf) {
}
}
+ private static Object convertLiteral(Object literal, TransformSpec
transform) {
+ if (transform == null) {
+ return literal;
+ }
+ try {
+ switch (transform.getTransformType()) {
+ case YEAR:
+ return parseYearToTransformYear(literal.toString());
+ case MONTH:
+ return parseMonthToTransformMonth(literal.toString());
+ case DAY:
+ return parseDayToTransformMonth(literal.toString());
+ case HOUR:
+ return parseHourToTransformHour(literal.toString());
+ case TRUNCATE:
+ return
Transforms.truncate(transform.getTransformParam().get()).bind(Types.StringType.get())
+ .apply(literal.toString());
+ case BUCKET:
+ return
Transforms.bucket(transform.getTransformParam().get()).bind(Types.StringType.get())
+ .apply(literal.toString());
+ case IDENTITY:
+ return literal;
+ default:
+ throw new UnsupportedOperationException("Unknown transform: " +
transform.getTransformType());
+ }
+ } catch (NumberFormatException | DateTimeException | IllegalStateException
e) {
+ throw new RuntimeException(
+ String.format("Unable to parse value '%s' as '%s' transform value",
literal.toString(), transform));
+ }
+ }
+
+ private static final int ICEBERG_EPOCH_YEAR = 1970;
+ private static final int ICEBERG_EPOCH_MONTH = 1;
+
+ /**
+ * In the partition path years are represented naturally, e.g. 1984.
However, we need
+ * to convert it to an integer which represents the years from 1970. So, for
1984 the
+ * return value should be 14.
+ */
+ private static Integer parseYearToTransformYear(String yearStr) {
+ int year = Integer.parseInt(yearStr);
+ return year - ICEBERG_EPOCH_YEAR;
+ }
+
+ /**
+ * In the partition path months are represented as year-month, e.g. 2021-01.
We
+ * need to convert it to a single integer which represents the months from
'1970-01'.
+ */
+ private static Integer parseMonthToTransformMonth(String monthStr) {
+ String[] parts = monthStr.split("-", -1);
+ Preconditions.checkState(parts.length == 2);
+ int year = Integer.parseInt(parts[0]);
+ int month = Integer.parseInt(parts[1]);
+ int years = year - ICEBERG_EPOCH_YEAR;
+ int months = month - ICEBERG_EPOCH_MONTH;
+ return years * 12 + months;
+ }
+
+ /**
+ * In the partition path days are represented as year-month-day, e.g.
2023-12-12.
+ * This functions converts this string to an integer which represents the
days from
+ * '1970-01-01' with the help of Iceberg's type converter.
+ */
+ private static Integer parseDayToTransformMonth(String monthStr) {
+ Literal<Integer> days = Literal.of(monthStr).to(Types.DateType.get());
+ return days.value();
+ }
+
+ /**
+ * In the partition path hours are represented as year-month-day-hour, e.g.
+ * 1970-01-01-01. We need to convert it to a single integer which represents
the hours
+ * from '1970-01-01 00:00:00'.
+ */
+ private static Integer parseHourToTransformHour(String hourStr) {
+ final OffsetDateTime epoch =
Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
+ String[] parts = hourStr.split("-", -1);
+ Preconditions.checkState(parts.length == 4);
+ int year = Integer.parseInt(parts[0]);
+ int month = Integer.parseInt(parts[1]);
+ int day = Integer.parseInt(parts[2]);
+ int hour = Integer.parseInt(parts[3]);
+ OffsetDateTime datetime = OffsetDateTime.of(LocalDateTime.of(year, month,
day, hour, /* minute=*/0),
+ ZoneOffset.UTC);
+ return (int) ChronoUnit.HOURS.between(epoch, datetime);
+ }
+
private static List<Object> leafToLiteralList(PredicateLeaf leaf) {
switch (leaf.getType()) {
case LONG:
diff --git
a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java
b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java
index 9d41e636dfe..45b83911076 100644
---
a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java
+++
b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java
@@ -36,6 +36,7 @@
import java.util.function.Function;
import java.util.stream.Collectors;
import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.lang3.ObjectUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -111,6 +112,8 @@
import org.apache.iceberg.expressions.Expression;
import org.apache.iceberg.expressions.Expressions;
import org.apache.iceberg.expressions.ResidualEvaluator;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.expressions.UnboundTerm;
import org.apache.iceberg.hive.CachedClientPool;
import org.apache.iceberg.hive.HiveSchemaUtil;
import org.apache.iceberg.hive.HiveTableOperations;
@@ -1130,18 +1133,12 @@ public void
preDropPartitions(org.apache.hadoop.hive.metastore.api.Table hmsTabl
List<org.apache.commons.lang3.tuple.Pair<Integer, byte[]>> partExprs)
throws MetaException {
Table icebergTbl = IcebergTableUtil.getTable(conf, hmsTable);
- Map<String, PartitionField> partitionFieldMap =
-
icebergTbl.spec().fields().stream().collect(Collectors.toMap(PartitionField::name,
Function.identity()));
DeleteFiles deleteFiles = icebergTbl.newDelete();
List<Expression> expressions = partExprs.stream().map(partExpr -> {
ExprNodeDesc exprNodeDesc = SerializationUtilities
.deserializeObjectWithTypeInformation(partExpr.getRight(), true);
SearchArgument sarg = ConvertAstToSearchArg.create(conf,
(ExprNodeGenericFuncDesc) exprNodeDesc);
- for (PredicateLeaf leaf : sarg.getLeaves()) {
- if (leaf.getColumnName() != null &&
!partitionFieldMap.containsKey(leaf.getColumnName())) {
- throw new UnsupportedOperationException("Drop Partition not
supported on Transformed Columns");
- }
- }
+ validatePartitionSpec(sarg, icebergTbl.spec());
return HiveIcebergFilterFactory.generateFilterExpression(sarg);
}).collect(Collectors.toList());
PartitionsTable partitionsTable = (PartitionsTable) MetadataTableUtils
@@ -1167,8 +1164,8 @@ public void
preDropPartitions(org.apache.hadoop.hive.metastore.api.Table hmsTabl
Expression partFilter = Expressions.alwaysTrue();
for (int index = 0; index < pSpec.fields().size(); index++) {
PartitionField field = icebergTbl.spec().fields().get(index);
- partFilter = Expressions.and(
- partFilter, Expressions.equal(field.name(),
partitionData.get(index, Object.class)));
+ UnboundPredicate<Object> equal =
getPartitionPredicate(partitionData, field, index, icebergTbl.schema());
+ partFilter = Expressions.and(partFilter, equal);
}
partitionSetFilter = Expressions.or(partitionSetFilter, partFilter);
}
@@ -1181,6 +1178,35 @@ public void
preDropPartitions(org.apache.hadoop.hive.metastore.api.Table hmsTabl
context.putToProperties(HiveMetaStoreClient.SKIP_DROP_PARTITION, "true");
}
+ private static void validatePartitionSpec(SearchArgument sarg, PartitionSpec
partitionSpec) {
+ for (PredicateLeaf leaf : sarg.getLeaves()) {
+ TransformSpec transformSpec =
TransformSpec.fromStringWithColumnName(leaf.getColumnName());
+ Types.NestedField column =
partitionSpec.schema().findField(transformSpec.getColumnName());
+ PartitionField partitionColumn =
+ partitionSpec.fields().stream().filter(pf -> pf.sourceId() ==
column.fieldId()).findFirst().get();
+
+ if (!(partitionColumn.transform() == null &&
transformSpec.transformTypeString() == null) &&
+
!partitionColumn.transform().toString().equalsIgnoreCase(transformSpec.transformTypeString()))
{
+ throw new UnsupportedOperationException(
+ "Invalid transform for column: " + transformSpec.getColumnName() +
+ " Expected: " + partitionColumn.transform() +
+ " Found: " + transformSpec.getTransformType());
+ }
+ }
+ }
+
+ private static UnboundPredicate<Object> getPartitionPredicate(PartitionData
partitionData, PartitionField field,
+ int index, Schema schema) {
+ String columName = schema.findField(field.sourceId()).name();
+ TransformSpec transformSpec =
TransformSpec.fromString(field.transform().toString(), columName);
+
+ UnboundTerm<Object> partitionColumn =
+ ObjectUtils.defaultIfNull(HiveIcebergFilterFactory.toTerm(columName,
transformSpec),
+ Expressions.ref(field.name()));
+
+ return Expressions.equal(partitionColumn, partitionData.get(index,
Object.class));
+ }
+
private class PreAlterTableProperties {
private String tableLocation;
private String format;
diff --git
a/iceberg/iceberg-handler/src/test/queries/positive/iceberg_drop_partition_transform_column.q
b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_drop_partition_transform_column.q
new file mode 100644
index 00000000000..e09691f6ddb
--- /dev/null
+++
b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_drop_partition_transform_column.q
@@ -0,0 +1,119 @@
+-- SORT_QUERY_RESULTS
+CREATE TABLE drop_partition_days (
+sensor_id STRING,
+location_id STRING,
+reading_time TIMESTAMP,
+temperature DOUBLE,
+humidity DOUBLE
+)
+PARTITIONED BY SPEC (location_id, days(reading_time))
+STORED BY ICEBERG
+TBLPROPERTIES (
+'write.format.default'='parquet',
+'format-version'='2',
+'write.parquet.compression-codec'='gzip'
+);
+
+INSERT INTO drop_partition_days VALUES
+('sensor_001', 'loc_001', '2024-06-01 10:00:00', 22.5, 60.0),
+('sensor_002', 'loc_002', '2024-06-01 10:15:00', 23.0, 58.0),
+('sensor_001', 'loc_001', '2024-06-02 11:00:00', 22.8, 61.0);
+
+ALTER TABLE drop_partition_days DROP PARTITION (location_id = 'loc_002',
days(reading_time) = '2024-06-01');
+
+SELECT * FROM drop_partition_days;
+
+CREATE TABLE drop_partition_years (
+sensor_id STRING,
+location_id STRING,
+reading_time TIMESTAMP,
+temperature DOUBLE,
+humidity DOUBLE
+)
+PARTITIONED BY SPEC (location_id, years(reading_time))
+STORED BY ICEBERG
+TBLPROPERTIES (
+'write.format.default'='parquet',
+'format-version'='2',
+'write.parquet.compression-codec'='gzip'
+);
+
+INSERT INTO drop_partition_years VALUES
+('sensor_001', 'loc_001', '2024-06-01 10:00:00', 22.5, 60.0),
+('sensor_002', 'loc_002', '2024-06-01 10:15:00', 23.0, 58.0),
+('sensor_001', 'loc_001', '2024-06-02 11:00:00', 22.8, 61.0);
+
+ALTER TABLE drop_partition_years DROP PARTITION (location_id = 'loc_002',
years(reading_time) = '2024');
+
+SELECT * FROM drop_partition_years;
+
+INSERT INTO drop_partition_years VALUES
+('sensor_001', 'loc_003', '2021-06-01 10:00:00', 20.5, 40.0),
+('sensor_002', 'loc_003', '2022-06-01 10:15:00', 33.0, 58.0),
+('sensor_001', 'loc_003', '2024-06-02 11:00:00', 12.8, 43.0);
+
+ALTER TABLE drop_partition_years DROP PARTITION (location_id = 'loc_003',
years(reading_time) <= '2022');
+
+SELECT * FROM drop_partition_years;
+
+CREATE TABLE drop_partition_months (
+sensor_id STRING,
+location_id STRING,
+reading_time TIMESTAMP,
+temperature DOUBLE,
+humidity DOUBLE
+)
+PARTITIONED BY SPEC (location_id, months(reading_time))
+STORED BY ICEBERG
+TBLPROPERTIES (
+'write.format.default'='parquet',
+'format-version'='2',
+'write.parquet.compression-codec'='gzip'
+);
+
+INSERT INTO drop_partition_months VALUES
+('sensor_001', 'loc_001', '2024-06-01 10:00:00', 22.5, 60.0),
+('sensor_002', 'loc_002', '2024-06-01 10:15:00', 23.0, 58.0),
+('sensor_001', 'loc_001', '2024-08-02 11:00:00', 22.8, 61.0);
+
+ALTER TABLE drop_partition_months DROP PARTITION (location_id = 'loc_002',
months(reading_time) >= '2024-06');
+
+SELECT * FROM drop_partition_months;
+
+CREATE TABLE drop_partition_hours (
+sensor_id STRING,
+location_id STRING,
+reading_time TIMESTAMP,
+temperature DOUBLE,
+humidity DOUBLE
+)
+PARTITIONED BY SPEC (location_id, hours(reading_time))
+STORED BY ICEBERG
+TBLPROPERTIES (
+'write.format.default'='parquet',
+'format-version'='2',
+'write.parquet.compression-codec'='gzip'
+);
+
+INSERT INTO drop_partition_hours VALUES
+('sensor_001', 'loc_001', '2024-06-01 10:00:00', 22.5, 60.0),
+('sensor_002', 'loc_002', '2024-06-01 10:15:00', 23.0, 58.0),
+('sensor_001', 'loc_001', '2024-06-02 11:00:00', 22.8, 61.0);
+
+ALTER TABLE drop_partition_hours DROP PARTITION (location_id = 'loc_002',
hours(reading_time) = '2024-06-01-10');
+
+SELECT * FROM drop_partition_hours;
+
+create external table drop_partition_truncate(a int, b string, c int)
partitioned by spec (truncate(3, b)) stored by iceberg;
+insert into drop_partition_truncate values (1, 'one', 50), (2, 'two', 51), (3,
'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one', 55), (333, 'two',
56);
+
+ALTER TABLE drop_partition_truncate DROP PARTITION (truncate(3, b) = 'one');
+
+SELECT * FROM drop_partition_truncate;
+
+create external table drop_partition_bucket(a int, b string, c int)
partitioned by spec (bucket(3,b)) stored by iceberg;
+insert into drop_partition_bucket values (1, 'one', 50), (2, 'two', 51), (3,
'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one', 55), (333, 'two',
56);
+
+ALTER TABLE drop_partition_bucket DROP PARTITION (bucket(3, b) = '0');
+
+SELECT * FROM drop_partition_bucket;
\ No newline at end of file
diff --git
a/iceberg/iceberg-handler/src/test/results/negative/iceberg_drop_partition_tranform_column.q.out
b/iceberg/iceberg-handler/src/test/results/negative/iceberg_drop_partition_tranform_column.q.out
index dee16170c04..27b2d294e83 100644
---
a/iceberg/iceberg-handler/src/test/results/negative/iceberg_drop_partition_tranform_column.q.out
+++
b/iceberg/iceberg-handler/src/test/results/negative/iceberg_drop_partition_tranform_column.q.out
@@ -50,4 +50,4 @@ PREHOOK: query: ALTER TABLE drop_partition DROP PARTITION
(location_id = 'loc_00
PREHOOK: type: ALTERTABLE_DROPPARTS
PREHOOK: Input: default@drop_partition
PREHOOK: Output:
default@drop_partition@location_id=loc_002/reading_time_day=2024-06-01
-FAILED: Execution Error, return code 40000 from
org.apache.hadoop.hive.ql.ddl.DDLTask. Drop Partition not supported on
Transformed Columns
+FAILED: Execution Error, return code 40000 from
org.apache.hadoop.hive.ql.ddl.DDLTask. Invalid transform for column:
reading_time Expected: day Found: IDENTITY
diff --git
a/iceberg/iceberg-handler/src/test/results/positive/iceberg_drop_partition_transform_column.q.out
b/iceberg/iceberg-handler/src/test/results/positive/iceberg_drop_partition_transform_column.q.out
new file mode 100644
index 00000000000..efb4da89944
--- /dev/null
+++
b/iceberg/iceberg-handler/src/test/results/positive/iceberg_drop_partition_transform_column.q.out
@@ -0,0 +1,371 @@
+PREHOOK: query: CREATE TABLE drop_partition_days (
+sensor_id STRING,
+location_id STRING,
+reading_time TIMESTAMP,
+temperature DOUBLE,
+humidity DOUBLE
+)
+PARTITIONED BY SPEC (location_id, days(reading_time))
+STORED BY ICEBERG
+TBLPROPERTIES (
+'write.format.default'='parquet',
+'format-version'='2',
+'write.parquet.compression-codec'='gzip'
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@drop_partition_days
+POSTHOOK: query: CREATE TABLE drop_partition_days (
+sensor_id STRING,
+location_id STRING,
+reading_time TIMESTAMP,
+temperature DOUBLE,
+humidity DOUBLE
+)
+PARTITIONED BY SPEC (location_id, days(reading_time))
+STORED BY ICEBERG
+TBLPROPERTIES (
+'write.format.default'='parquet',
+'format-version'='2',
+'write.parquet.compression-codec'='gzip'
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@drop_partition_days
+PREHOOK: query: INSERT INTO drop_partition_days VALUES
+('sensor_001', 'loc_001', '2024-06-01 10:00:00', 22.5, 60.0),
+('sensor_002', 'loc_002', '2024-06-01 10:15:00', 23.0, 58.0),
+('sensor_001', 'loc_001', '2024-06-02 11:00:00', 22.8, 61.0)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@drop_partition_days
+POSTHOOK: query: INSERT INTO drop_partition_days VALUES
+('sensor_001', 'loc_001', '2024-06-01 10:00:00', 22.5, 60.0),
+('sensor_002', 'loc_002', '2024-06-01 10:15:00', 23.0, 58.0),
+('sensor_001', 'loc_001', '2024-06-02 11:00:00', 22.8, 61.0)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@drop_partition_days
+PREHOOK: query: ALTER TABLE drop_partition_days DROP PARTITION (location_id =
'loc_002', days(reading_time) = '2024-06-01')
+PREHOOK: type: ALTERTABLE_DROPPARTS
+PREHOOK: Input: default@drop_partition_days
+PREHOOK: Output:
default@drop_partition_days@location_id=loc_002/reading_time_day=2024-06-01
+POSTHOOK: query: ALTER TABLE drop_partition_days DROP PARTITION (location_id =
'loc_002', days(reading_time) = '2024-06-01')
+POSTHOOK: type: ALTERTABLE_DROPPARTS
+POSTHOOK: Input: default@drop_partition_days
+POSTHOOK: Output:
default@drop_partition_days@location_id=loc_002/reading_time_day=2024-06-01
+PREHOOK: query: SELECT * FROM drop_partition_days
+PREHOOK: type: QUERY
+PREHOOK: Input: default@drop_partition_days
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: SELECT * FROM drop_partition_days
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@drop_partition_days
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+sensor_001 loc_001 2024-06-01 10:00:00 22.5 60.0
+sensor_001 loc_001 2024-06-02 11:00:00 22.8 61.0
+PREHOOK: query: CREATE TABLE drop_partition_years (
+sensor_id STRING,
+location_id STRING,
+reading_time TIMESTAMP,
+temperature DOUBLE,
+humidity DOUBLE
+)
+PARTITIONED BY SPEC (location_id, years(reading_time))
+STORED BY ICEBERG
+TBLPROPERTIES (
+'write.format.default'='parquet',
+'format-version'='2',
+'write.parquet.compression-codec'='gzip'
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@drop_partition_years
+POSTHOOK: query: CREATE TABLE drop_partition_years (
+sensor_id STRING,
+location_id STRING,
+reading_time TIMESTAMP,
+temperature DOUBLE,
+humidity DOUBLE
+)
+PARTITIONED BY SPEC (location_id, years(reading_time))
+STORED BY ICEBERG
+TBLPROPERTIES (
+'write.format.default'='parquet',
+'format-version'='2',
+'write.parquet.compression-codec'='gzip'
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@drop_partition_years
+PREHOOK: query: INSERT INTO drop_partition_years VALUES
+('sensor_001', 'loc_001', '2024-06-01 10:00:00', 22.5, 60.0),
+('sensor_002', 'loc_002', '2024-06-01 10:15:00', 23.0, 58.0),
+('sensor_001', 'loc_001', '2024-06-02 11:00:00', 22.8, 61.0)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@drop_partition_years
+POSTHOOK: query: INSERT INTO drop_partition_years VALUES
+('sensor_001', 'loc_001', '2024-06-01 10:00:00', 22.5, 60.0),
+('sensor_002', 'loc_002', '2024-06-01 10:15:00', 23.0, 58.0),
+('sensor_001', 'loc_001', '2024-06-02 11:00:00', 22.8, 61.0)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@drop_partition_years
+PREHOOK: query: ALTER TABLE drop_partition_years DROP PARTITION (location_id =
'loc_002', years(reading_time) = '2024')
+PREHOOK: type: ALTERTABLE_DROPPARTS
+PREHOOK: Input: default@drop_partition_years
+PREHOOK: Output:
default@drop_partition_years@location_id=loc_002/reading_time_year=2024
+POSTHOOK: query: ALTER TABLE drop_partition_years DROP PARTITION (location_id
= 'loc_002', years(reading_time) = '2024')
+POSTHOOK: type: ALTERTABLE_DROPPARTS
+POSTHOOK: Input: default@drop_partition_years
+POSTHOOK: Output:
default@drop_partition_years@location_id=loc_002/reading_time_year=2024
+PREHOOK: query: SELECT * FROM drop_partition_years
+PREHOOK: type: QUERY
+PREHOOK: Input: default@drop_partition_years
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: SELECT * FROM drop_partition_years
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@drop_partition_years
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+sensor_001 loc_001 2024-06-01 10:00:00 22.5 60.0
+sensor_001 loc_001 2024-06-02 11:00:00 22.8 61.0
+PREHOOK: query: INSERT INTO drop_partition_years VALUES
+('sensor_001', 'loc_003', '2021-06-01 10:00:00', 20.5, 40.0),
+('sensor_002', 'loc_003', '2022-06-01 10:15:00', 33.0, 58.0),
+('sensor_001', 'loc_003', '2024-06-02 11:00:00', 12.8, 43.0)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@drop_partition_years
+POSTHOOK: query: INSERT INTO drop_partition_years VALUES
+('sensor_001', 'loc_003', '2021-06-01 10:00:00', 20.5, 40.0),
+('sensor_002', 'loc_003', '2022-06-01 10:15:00', 33.0, 58.0),
+('sensor_001', 'loc_003', '2024-06-02 11:00:00', 12.8, 43.0)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@drop_partition_years
+PREHOOK: query: ALTER TABLE drop_partition_years DROP PARTITION (location_id =
'loc_003', years(reading_time) <= '2022')
+PREHOOK: type: ALTERTABLE_DROPPARTS
+PREHOOK: Input: default@drop_partition_years
+PREHOOK: Output:
default@drop_partition_years@location_id=loc_003/reading_time_year=2021
+PREHOOK: Output:
default@drop_partition_years@location_id=loc_003/reading_time_year=2022
+POSTHOOK: query: ALTER TABLE drop_partition_years DROP PARTITION (location_id
= 'loc_003', years(reading_time) <= '2022')
+POSTHOOK: type: ALTERTABLE_DROPPARTS
+POSTHOOK: Input: default@drop_partition_years
+POSTHOOK: Output:
default@drop_partition_years@location_id=loc_003/reading_time_year=2021
+POSTHOOK: Output:
default@drop_partition_years@location_id=loc_003/reading_time_year=2022
+PREHOOK: query: SELECT * FROM drop_partition_years
+PREHOOK: type: QUERY
+PREHOOK: Input: default@drop_partition_years
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: SELECT * FROM drop_partition_years
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@drop_partition_years
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+sensor_001 loc_001 2024-06-01 10:00:00 22.5 60.0
+sensor_001 loc_001 2024-06-02 11:00:00 22.8 61.0
+sensor_001 loc_003 2024-06-02 11:00:00 12.8 43.0
+PREHOOK: query: CREATE TABLE drop_partition_months (
+sensor_id STRING,
+location_id STRING,
+reading_time TIMESTAMP,
+temperature DOUBLE,
+humidity DOUBLE
+)
+PARTITIONED BY SPEC (location_id, months(reading_time))
+STORED BY ICEBERG
+TBLPROPERTIES (
+'write.format.default'='parquet',
+'format-version'='2',
+'write.parquet.compression-codec'='gzip'
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@drop_partition_months
+POSTHOOK: query: CREATE TABLE drop_partition_months (
+sensor_id STRING,
+location_id STRING,
+reading_time TIMESTAMP,
+temperature DOUBLE,
+humidity DOUBLE
+)
+PARTITIONED BY SPEC (location_id, months(reading_time))
+STORED BY ICEBERG
+TBLPROPERTIES (
+'write.format.default'='parquet',
+'format-version'='2',
+'write.parquet.compression-codec'='gzip'
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@drop_partition_months
+PREHOOK: query: INSERT INTO drop_partition_months VALUES
+('sensor_001', 'loc_001', '2024-06-01 10:00:00', 22.5, 60.0),
+('sensor_002', 'loc_002', '2024-06-01 10:15:00', 23.0, 58.0),
+('sensor_001', 'loc_001', '2024-08-02 11:00:00', 22.8, 61.0)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@drop_partition_months
+POSTHOOK: query: INSERT INTO drop_partition_months VALUES
+('sensor_001', 'loc_001', '2024-06-01 10:00:00', 22.5, 60.0),
+('sensor_002', 'loc_002', '2024-06-01 10:15:00', 23.0, 58.0),
+('sensor_001', 'loc_001', '2024-08-02 11:00:00', 22.8, 61.0)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@drop_partition_months
+PREHOOK: query: ALTER TABLE drop_partition_months DROP PARTITION (location_id
= 'loc_002', months(reading_time) >= '2024-06')
+PREHOOK: type: ALTERTABLE_DROPPARTS
+PREHOOK: Input: default@drop_partition_months
+PREHOOK: Output:
default@drop_partition_months@location_id=loc_002/reading_time_month=2024-06
+POSTHOOK: query: ALTER TABLE drop_partition_months DROP PARTITION (location_id
= 'loc_002', months(reading_time) >= '2024-06')
+POSTHOOK: type: ALTERTABLE_DROPPARTS
+POSTHOOK: Input: default@drop_partition_months
+POSTHOOK: Output:
default@drop_partition_months@location_id=loc_002/reading_time_month=2024-06
+PREHOOK: query: SELECT * FROM drop_partition_months
+PREHOOK: type: QUERY
+PREHOOK: Input: default@drop_partition_months
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: SELECT * FROM drop_partition_months
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@drop_partition_months
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+sensor_001 loc_001 2024-06-01 10:00:00 22.5 60.0
+sensor_001 loc_001 2024-08-02 11:00:00 22.8 61.0
+PREHOOK: query: CREATE TABLE drop_partition_hours (
+sensor_id STRING,
+location_id STRING,
+reading_time TIMESTAMP,
+temperature DOUBLE,
+humidity DOUBLE
+)
+PARTITIONED BY SPEC (location_id, hours(reading_time))
+STORED BY ICEBERG
+TBLPROPERTIES (
+'write.format.default'='parquet',
+'format-version'='2',
+'write.parquet.compression-codec'='gzip'
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@drop_partition_hours
+POSTHOOK: query: CREATE TABLE drop_partition_hours (
+sensor_id STRING,
+location_id STRING,
+reading_time TIMESTAMP,
+temperature DOUBLE,
+humidity DOUBLE
+)
+PARTITIONED BY SPEC (location_id, hours(reading_time))
+STORED BY ICEBERG
+TBLPROPERTIES (
+'write.format.default'='parquet',
+'format-version'='2',
+'write.parquet.compression-codec'='gzip'
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@drop_partition_hours
+PREHOOK: query: INSERT INTO drop_partition_hours VALUES
+('sensor_001', 'loc_001', '2024-06-01 10:00:00', 22.5, 60.0),
+('sensor_002', 'loc_002', '2024-06-01 10:15:00', 23.0, 58.0),
+('sensor_001', 'loc_001', '2024-06-02 11:00:00', 22.8, 61.0)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@drop_partition_hours
+POSTHOOK: query: INSERT INTO drop_partition_hours VALUES
+('sensor_001', 'loc_001', '2024-06-01 10:00:00', 22.5, 60.0),
+('sensor_002', 'loc_002', '2024-06-01 10:15:00', 23.0, 58.0),
+('sensor_001', 'loc_001', '2024-06-02 11:00:00', 22.8, 61.0)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@drop_partition_hours
+PREHOOK: query: ALTER TABLE drop_partition_hours DROP PARTITION (location_id =
'loc_002', hours(reading_time) = '2024-06-01-10')
+PREHOOK: type: ALTERTABLE_DROPPARTS
+PREHOOK: Input: default@drop_partition_hours
+PREHOOK: Output:
default@drop_partition_hours@location_id=loc_002/reading_time_hour=2024-06-01-10
+POSTHOOK: query: ALTER TABLE drop_partition_hours DROP PARTITION (location_id
= 'loc_002', hours(reading_time) = '2024-06-01-10')
+POSTHOOK: type: ALTERTABLE_DROPPARTS
+POSTHOOK: Input: default@drop_partition_hours
+POSTHOOK: Output:
default@drop_partition_hours@location_id=loc_002/reading_time_hour=2024-06-01-10
+PREHOOK: query: SELECT * FROM drop_partition_hours
+PREHOOK: type: QUERY
+PREHOOK: Input: default@drop_partition_hours
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: SELECT * FROM drop_partition_hours
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@drop_partition_hours
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+sensor_001 loc_001 2024-06-01 10:00:00 22.5 60.0
+sensor_001 loc_001 2024-06-02 11:00:00 22.8 61.0
+PREHOOK: query: create external table drop_partition_truncate(a int, b string,
c int) partitioned by spec (truncate(3, b)) stored by iceberg
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@drop_partition_truncate
+POSTHOOK: query: create external table drop_partition_truncate(a int, b
string, c int) partitioned by spec (truncate(3, b)) stored by iceberg
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@drop_partition_truncate
+PREHOOK: query: insert into drop_partition_truncate values (1, 'one', 50), (2,
'two', 51), (3, 'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one',
55), (333, 'two', 56)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@drop_partition_truncate
+POSTHOOK: query: insert into drop_partition_truncate values (1, 'one', 50),
(2, 'two', 51), (3, 'three', 52), (4, 'four', 53), (5, 'five', 54), (111,
'one', 55), (333, 'two', 56)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@drop_partition_truncate
+PREHOOK: query: ALTER TABLE drop_partition_truncate DROP PARTITION
(truncate(3, b) = 'one')
+PREHOOK: type: ALTERTABLE_DROPPARTS
+PREHOOK: Input: default@drop_partition_truncate
+PREHOOK: Output: default@drop_partition_truncate@b_trunc=one
+POSTHOOK: query: ALTER TABLE drop_partition_truncate DROP PARTITION
(truncate(3, b) = 'one')
+POSTHOOK: type: ALTERTABLE_DROPPARTS
+POSTHOOK: Input: default@drop_partition_truncate
+POSTHOOK: Output: default@drop_partition_truncate@b_trunc=one
+PREHOOK: query: SELECT * FROM drop_partition_truncate
+PREHOOK: type: QUERY
+PREHOOK: Input: default@drop_partition_truncate
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: SELECT * FROM drop_partition_truncate
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@drop_partition_truncate
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+2 two 51
+3 three 52
+333 two 56
+4 four 53
+5 five 54
+PREHOOK: query: create external table drop_partition_bucket(a int, b string, c
int) partitioned by spec (bucket(3,b)) stored by iceberg
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@drop_partition_bucket
+POSTHOOK: query: create external table drop_partition_bucket(a int, b string,
c int) partitioned by spec (bucket(3,b)) stored by iceberg
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@drop_partition_bucket
+PREHOOK: query: insert into drop_partition_bucket values (1, 'one', 50), (2,
'two', 51), (3, 'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one',
55), (333, 'two', 56)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@drop_partition_bucket
+POSTHOOK: query: insert into drop_partition_bucket values (1, 'one', 50), (2,
'two', 51), (3, 'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one',
55), (333, 'two', 56)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@drop_partition_bucket
+PREHOOK: query: ALTER TABLE drop_partition_bucket DROP PARTITION (bucket(3, b)
= '0')
+PREHOOK: type: ALTERTABLE_DROPPARTS
+PREHOOK: Input: default@drop_partition_bucket
+PREHOOK: Output: default@drop_partition_bucket@b_bucket=1
+POSTHOOK: query: ALTER TABLE drop_partition_bucket DROP PARTITION (bucket(3,
b) = '0')
+POSTHOOK: type: ALTERTABLE_DROPPARTS
+POSTHOOK: Input: default@drop_partition_bucket
+POSTHOOK: Output: default@drop_partition_bucket@b_bucket=1
+PREHOOK: query: SELECT * FROM drop_partition_bucket
+PREHOOK: type: QUERY
+PREHOOK: Input: default@drop_partition_bucket
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: SELECT * FROM drop_partition_bucket
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@drop_partition_bucket
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+1 one 50
+111 one 55
+5 five 54
diff --git
a/parser/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
b/parser/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
index cef19520f84..7dad9dcdc31 100644
--- a/parser/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
+++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
@@ -821,6 +821,7 @@ partitionSpec
partitionVal
:
identifier (EQUAL constant)? -> ^(TOK_PARTVAL identifier constant?)
+ | functionExpr (EQUAL constant)? -> ^(TOK_PARTVAL functionExpr constant?)
;
partitionSelectorSpec
@@ -831,8 +832,25 @@ partitionSelectorSpec
partitionSelectorVal
:
identifier partitionSelectorOperator constant -> ^(TOK_PARTVAL identifier
partitionSelectorOperator constant)
+ | functionExpr partitionSelectorOperator constant -> ^(TOK_PARTVAL
functionExpr partitionSelectorOperator constant)
;
+functionExpr
+ :
+ funcName LPAREN identifier RPAREN -> ^(TOK_FUNCTION funcName identifier)
+ | KW_TRUNCATE LPAREN width=Number COMMA identifier RPAREN ->
^(TOK_FUNCTION KW_TRUNCATE $width identifier)
+ | KW_BUCKET LPAREN width=Number COMMA identifier RPAREN -> ^(TOK_FUNCTION
KW_BUCKET $width identifier)
+ ;
+
+funcName
+ :
+ KW_DAY
+ | KW_MONTH
+ | KW_YEAR
+ | KW_HOUR
+ ;
+
+
partitionSelectorOperator
:
KW_LIKE | subQuerySelectorOperator
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java
b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java
index 186f2481d3d..9964b936906 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java
@@ -40,6 +40,7 @@
import org.antlr.runtime.tree.Tree;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.util.Pair;
+import org.apache.commons.lang3.ObjectUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hive.common.type.HiveDecimal;
import org.apache.hadoop.hive.conf.HiveConf;
@@ -325,7 +326,7 @@ public static boolean containsTokenOfType(ASTNode root,
PTFUtils.Predicate<ASTNo
}
private static void handleSetColRefs(ASTNode tree, Context ctx) {
- CalcitePlanner.ASTSearcher astSearcher = new CalcitePlanner.ASTSearcher();
+ ASTSearcher astSearcher = new ASTSearcher();
while (true) {
astSearcher.reset();
ASTNode setCols = astSearcher.depthFirstSearch(tree,
HiveParser.TOK_SETCOLREF);
@@ -580,7 +581,7 @@ public static Map<Integer, List<ExprNodeGenericFuncDesc>>
getFullPartitionSpecs(
CommonTree ast, Table table, Configuration conf, boolean canGroupExprs)
throws SemanticException {
String defaultPartitionName = HiveConf.getVar(conf,
HiveConf.ConfVars.DEFAULT_PARTITION_NAME);
Map<String, String> colTypes = new HashMap<>();
- List<FieldSchema> partitionKeys = table.hasNonNativePartitionSupport() ?
+ List<FieldSchema> partitionKeys = table.hasNonNativePartitionSupport() ?
table.getStorageHandler().getPartitionKeys(table) :
table.getPartitionKeys();
for (FieldSchema fs : partitionKeys) {
colTypes.put(fs.getName().toLowerCase(), fs.getType());
@@ -598,7 +599,24 @@ public static Map<Integer, List<ExprNodeGenericFuncDesc>>
getFullPartitionSpecs(
for (int i = 0; i < partSpecTree.getChildCount(); ++i) {
CommonTree partSpecSingleKey = (CommonTree) partSpecTree.getChild(i);
assert (partSpecSingleKey.getType() == HiveParser.TOK_PARTVAL);
- String key =
stripIdentifierQuotes(partSpecSingleKey.getChild(0).getText()).toLowerCase();
+ String transform = null;
+ String key;
+ Tree partitionTree = partSpecSingleKey.getChild(0);
+ if (partitionTree.getType() == HiveParser.TOK_FUNCTION) {
+ int childCount = partitionTree.getChildCount();
+ if (childCount == 2) { // Case with unary function
+ key =
stripIdentifierQuotes(partitionTree.getChild(1).getText()).toLowerCase();
+ transform = partitionTree.getChild(0).getText().toLowerCase() +
"(" + key + ")";
+ } else if (childCount == 3) { // Case with transform, columnName,
and integer
+ key =
stripIdentifierQuotes(partitionTree.getChild(2).getText()).toLowerCase();
+ String transformParam = partitionTree.getChild(1).getText();
+ transform = partitionTree.getChild(0).getText().toLowerCase() +
"(" + transformParam + ", " + key + ")";
+ } else {
+ throw new SemanticException("Unexpected number of children in
partition spec");
+ }
+ } else {
+ key = stripIdentifierQuotes(partitionTree.getText()).toLowerCase();
+ }
String operator = partSpecSingleKey.getChild(1).getText();
ASTNode partValNode = (ASTNode)partSpecSingleKey.getChild(2);
TypeCheckCtx typeCheckCtx = new TypeCheckCtx(null);
@@ -608,7 +626,7 @@ public static Map<Integer, List<ExprNodeGenericFuncDesc>>
getFullPartitionSpecs(
boolean isDefaultPartitionName = val.equals(defaultPartitionName);
- String type = colTypes.get(key);
+ String type = transform != null ? "string" : colTypes.get(key);
if (type == null) {
throw new SemanticException("Column " + key + " is not a partition
key");
}
@@ -623,7 +641,7 @@ public static Map<Integer, List<ExprNodeGenericFuncDesc>>
getFullPartitionSpecs(
}
}
- ExprNodeColumnDesc column = new ExprNodeColumnDesc(pti, key, null,
true);
+ ExprNodeColumnDesc column = new ExprNodeColumnDesc(pti,
ObjectUtils.defaultIfNull(transform, key), null, true);
ExprNodeGenericFuncDesc op;
if (!isDefaultPartitionName) {
op = PartitionUtils.makeBinaryPredicate(operator, column, new
ExprNodeConstantDesc(pti, val));
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TransformSpec.java
b/ql/src/java/org/apache/hadoop/hive/ql/parse/TransformSpec.java
index 6008a5f9a9e..625924dfb2a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TransformSpec.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TransformSpec.java
@@ -66,6 +66,16 @@ public void setTransformParam(Optional<Integer>
transformParam) {
this.transformParam = transformParam;
}
+ public String transformTypeString() {
+ if (transformType == null) {
+ return null;
+ }
+ if (transformParam.isPresent()) {
+ return transformType.name() + "[" + transformParam.get() + "]";
+ }
+ return transformType.name();
+ }
+
public static TransformType fromString(String transformString) {
Matcher widthMatcher = HAS_WIDTH.matcher(transformString);
if (widthMatcher.matches()) {
@@ -73,4 +83,46 @@ public static TransformType fromString(String
transformString) {
}
return TransformType.valueOf(transformString.toUpperCase(Locale.ROOT));
}
+
+ public static TransformSpec fromString(String transfromString, String
columnName) {
+ Matcher widthMatcher = HAS_WIDTH.matcher(transfromString);
+ Optional<Integer> width = Optional.empty();
+ if (widthMatcher.matches()) {
+ transfromString = widthMatcher.group(1);
+ width = Optional.of(Integer.parseInt(widthMatcher.group(2)));
+ return new TransformSpec(columnName,
TransformType.valueOf(transfromString.toUpperCase(Locale.ROOT)), width);
+ }
+ return new TransformSpec(columnName,
TransformType.valueOf(transfromString.toUpperCase(Locale.ROOT)), width);
+ }
+
+ public static TransformSpec fromStringWithColumnName(String transformString)
{
+ if (transformString == null || !transformString.contains("(")) {
+ return new TransformSpec(transformString, TransformType.IDENTITY,
Optional.empty());
+ }
+ transformString = transformString.trim();
+
+ // Extract transform type
+ String transformName =
transformString.split("\\(")[0].toLowerCase(Locale.ROOT);
+ String innerContent =
transformString.split("\\(")[1].split("\\)")[0].trim();
+
+ // Normalize transform name (convert "years" -> "year", "months" ->
"month", etc.)
+ transformName =
+ transformName.endsWith("s") ? transformName.substring(0,
transformName.length() - 1) : transformName;
+
+ // Handle transforms with width (truncate, bucket)
+ if (transformName.equals("truncate") || transformName.equals("bucket")) {
+ String[] parts = innerContent.split(",");
+ if (parts.length != 2) {
+ throw new IllegalArgumentException("Invalid format for " +
transformName + ": " + transformString);
+ }
+ int width = Integer.parseInt(parts[0].trim()); // First is width
+ String columnName = parts[1].trim(); // Second is column
+ return new TransformSpec(columnName,
TransformType.valueOf(transformName.toUpperCase(Locale.ROOT)),
+ Optional.of(width));
+ }
+
+ // Handle other cases (year, month, day, hour)
+ return new TransformSpec(innerContent,
TransformType.valueOf(transformName.toUpperCase(Locale.ROOT)),
+ Optional.empty());
+ }
}