Description
When using partition projection in the Hive connector, DATE partition values are parsed with yyyy-M-d
rather than the configured partition_projection_format
.
To reproduce:
Create a text file with the following content:
a1 b2 c3
d4 e5 f6
g7 h8 i9
Upload the file to an S3 bucket my-bucket
(you'll need to change this) with the path /logs/123456789012/us-west-001/metadaddy-test/2025/04/01/2025-04-01-21-32-16-E568B2907131C0C0
.
Create a table with partition projection:
CREATE TABLE regex_test(
foo VARCHAR,
bar VARCHAR,
baz VARCHAR,
timestamp DATE WITH (
partition_projection_format='yyyy/MM/dd',
partition_projection_interval=1,
partition_projection_interval_unit='DAYS',
partition_projection_range=ARRAY['NOW-365DAYS', 'NOW'],
partition_projection_type='date'
)
)
WITH (
format='REGEX',
partitioned_by=ARRAY['timestamp'],
regex='([^ ]*) ([^ ]*) \\[(.*?)\\]$',
external_location='s3://my-bucket/logs/123456789012/us-west-001/metadaddy-test',
partition_projection_enabled=true,
partition_projection_location_template='s3://my-bucket/logs/123456789012/us-west-001/metadaddy-test/${timestamp}'
);
Try to SELECT *
:
SELECT * FROM regex_test;
Query 20250422_010824_00035_tttig failed: Invalid partition value '2024/04/22' for DATE partition key: timestamp=2024%2F04%2F22
Stack trace:
2025-04-22T01:08:24.138Z DEBUG Query-20250422_010824_00035_tttig-494 io.trino.execution.QueryStateMachine Query 20250422_010824_00035_tttig failed
io.trino.spi.TrinoException: Invalid partition value '2024/04/22' for DATE partition key: timestamp=2024%2F04%2F22
at io.trino.plugin.hive.util.HiveUtil.datePartitionKey(HiveUtil.java:449)
at io.trino.plugin.hive.util.HiveUtil.parsePartitionValue(HiveUtil.java:313)
at io.trino.plugin.hive.HivePartitionManager.parsePartition(HivePartitionManager.java:265)
at io.trino.plugin.hive.HivePartitionManager.parseValuesAndFilterPartition(HivePartitionManager.java:216)
at io.trino.plugin.hive.HivePartitionManager.lambda$getPartitions$3(HivePartitionManager.java:113)
at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:215)
at java.base/java.util.Spliterators$ArraySpliterator.tryAdvance(Spliterators.java:1034)
at java.base/java.util.stream.StreamSpliterators$WrappingSpliterator.lambda$initPartialTraversalState$0(StreamSpliterators.java:297)
at java.base/java.util.stream.StreamSpliterators$AbstractWrappingSpliterator.fillBuffer(StreamSpliterators.java:206)
at java.base/java.util.stream.StreamSpliterators$AbstractWrappingSpliterator.doAdvance(StreamSpliterators.java:161)
at java.base/java.util.stream.StreamSpliterators$WrappingSpliterator.tryAdvance(StreamSpliterators.java:303)
at java.base/java.util.Spliterators$1Adapter.hasNext(Spliterators.java:669)
at io.trino.plugin.hive.HivePartitionManager.tryLoadPartitions(HivePartitionManager.java:199)
at io.trino.plugin.hive.HiveMetadata.lambda$getTableProperties$90(HiveMetadata.java:3044)
at java.base/java.util.Optional.or(Optional.java:313)
at io.trino.plugin.hive.HiveMetadata.getTableProperties(HiveMetadata.java:3039)
at io.trino.plugin.base.classloader.ClassLoaderSafeConnectorMetadata.getTableProperties(ClassLoaderSafeConnectorMetadata.java:992)
at io.trino.tracing.TracingConnectorMetadata.getTableProperties(TracingConnectorMetadata.java:1162)
at io.trino.metadata.MetadataManager.getTableProperties(MetadataManager.java:400)
at io.trino.tracing.TracingMetadata.getTableProperties(TracingMetadata.java:247)
at io.trino.sql.planner.planprinter.TableInfoSupplier.apply(TableInfoSupplier.java:47)
at io.trino.sql.planner.planprinter.TableInfoSupplier.apply(TableInfoSupplier.java:31)
at io.trino.sql.planner.planprinter.PlanPrinter$Visitor.visitScanFilterAndProjectInfo(PlanPrinter.java:1235)
at io.trino.sql.planner.planprinter.PlanPrinter$Visitor.visitProject(PlanPrinter.java:1203)
at io.trino.sql.planner.planprinter.PlanPrinter$Visitor.visitProject(PlanPrinter.java:641)
at io.trino.sql.planner.plan.ProjectNode.accept(ProjectNode.java:81)
at io.trino.sql.planner.planprinter.PlanPrinter$Visitor.visitScanFilterAndProjectInfo(PlanPrinter.java:1315)
at io.trino.sql.planner.planprinter.PlanPrinter$Visitor.visitProject(PlanPrinter.java:1203)
at io.trino.sql.planner.planprinter.PlanPrinter$Visitor.visitProject(PlanPrinter.java:641)
at io.trino.sql.planner.plan.ProjectNode.accept(ProjectNode.java:81)
at io.trino.sql.planner.planprinter.PlanPrinter$Visitor.visitScanFilterAndProjectInfo(PlanPrinter.java:1315)
at io.trino.sql.planner.planprinter.PlanPrinter$Visitor.visitProject(PlanPrinter.java:1203)
at io.trino.sql.planner.planprinter.PlanPrinter$Visitor.visitProject(PlanPrinter.java:641)
at io.trino.sql.planner.plan.ProjectNode.accept(ProjectNode.java:81)
at io.trino.sql.planner.planprinter.PlanPrinter$Visitor.visitScanFilterAndProjectInfo(PlanPrinter.java:1315)
at io.trino.sql.planner.planprinter.PlanPrinter$Visitor.visitProject(PlanPrinter.java:1203)
at io.trino.sql.planner.planprinter.PlanPrinter$Visitor.visitProject(PlanPrinter.java:641)
at io.trino.sql.planner.plan.ProjectNode.accept(ProjectNode.java:81)
at io.trino.sql.planner.planprinter.PlanPrinter$Visitor.processChildren(PlanPrinter.java:1963)
at io.trino.sql.planner.planprinter.PlanPrinter$Visitor.visitOutput(PlanPrinter.java:1444)
at io.trino.sql.planner.planprinter.PlanPrinter$Visitor.visitOutput(PlanPrinter.java:641)
at io.trino.sql.planner.plan.OutputNode.accept(OutputNode.java:82)
at io.trino.sql.planner.planprinter.PlanPrinter.<init>(PlanPrinter.java:240)
at io.trino.sql.planner.planprinter.PlanPrinter.textLogicalPlan(PlanPrinter.java:385)
at io.trino.sql.planner.planprinter.PlanPrinter.textLogicalPlan(PlanPrinter.java:362)
at io.trino.sql.planner.LogicalPlanner.plan(LogicalPlanner.java:255)
at io.trino.sql.planner.LogicalPlanner.plan(LogicalPlanner.java:244)
at io.trino.sql.planner.LogicalPlanner.plan(LogicalPlanner.java:239)
at io.trino.execution.SqlQueryExecution.doPlanQuery(SqlQueryExecution.java:503)
at io.trino.execution.SqlQueryExecution.planQuery(SqlQueryExecution.java:482)
at io.trino.execution.SqlQueryExecution.start(SqlQueryExecution.java:420)
at io.trino.execution.SqlQueryManager.createQuery(SqlQueryManager.java:272)
at io.trino.dispatcher.LocalDispatchQuery.startExecution(LocalDispatchQuery.java:150)
at io.trino.dispatcher.LocalDispatchQuery.lambda$waitForMinimumWorkers$2(LocalDispatchQuery.java:134)
at io.airlift.concurrent.MoreFutures.lambda$addSuccessCallback$12(MoreFutures.java:570)
at io.airlift.concurrent.MoreFutures$3.onSuccess(MoreFutures.java:545)
at com.google.common.util.concurrent.Futures$CallbackListener.run(Futures.java:1139)
at io.trino.$gen.Trino_474____20250422_003045_2.run(Unknown Source)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1095)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:619)
at java.base/java.lang.Thread.run(Thread.java:1447)
The problem seems to be that the Hive partition manager is (correctly) generating partition names of the form 2024/04/22
, but they are being parsed by HiveUtil.parseHiveDate
using HIVE_DATE_PARSER
, which is configured with date formats of the form yyyy-M-d
.
Metadata
Metadata
Assignees
Labels
No labels