Skip to content
Closed
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import io.trino.plugin.hive.metastore.HiveMetastoreFactory;
import io.trino.plugin.iceberg.util.FileOperationUtils;
import io.trino.plugin.tpch.TpchPlugin;
import io.trino.sql.planner.plan.FilterNode;
import io.trino.testing.AbstractTestQueryFramework;
import io.trino.testing.DistributedQueryRunner;
import io.trino.testing.QueryRunner;
Expand All @@ -35,13 +36,17 @@
import org.junit.jupiter.api.parallel.Execution;
import org.junit.jupiter.api.parallel.ExecutionMode;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.MethodSource;

import java.nio.file.Path;
import java.util.Optional;
import java.util.stream.Stream;

import static com.google.common.collect.ImmutableMultiset.toImmutableMultiset;
import static io.trino.SystemSessionProperties.MIN_INPUT_SIZE_PER_TASK;
import static io.trino.plugin.iceberg.IcebergFileFormat.AVRO;
import static io.trino.plugin.iceberg.IcebergFileFormat.ORC;
import static io.trino.plugin.iceberg.IcebergQueryRunner.ICEBERG_CATALOG;
import static io.trino.plugin.iceberg.IcebergSessionProperties.COLLECT_EXTENDED_STATISTICS_ON_WRITE;
import static io.trino.plugin.iceberg.IcebergTestUtils.getFileSystemFactory;
Expand All @@ -55,6 +60,9 @@
import static io.trino.plugin.iceberg.util.FileOperationUtils.FileType.STATS;
import static io.trino.plugin.iceberg.util.FileOperationUtils.Scope.ALL_FILES;
import static io.trino.plugin.iceberg.util.FileOperationUtils.Scope.METADATA_FILES;
import static io.trino.sql.planner.plan.TopNRankingNode.RankingType.DENSE_RANK;
import static io.trino.sql.planner.plan.TopNRankingNode.RankingType.RANK;
import static io.trino.sql.planner.plan.TopNRankingNode.RankingType.ROW_NUMBER;
import static io.trino.testing.MultisetAssertions.assertMultisetsEqual;
import static io.trino.testing.TestingNames.randomNameSuffix;
import static io.trino.testing.TestingSession.testSessionBuilder;
Expand Down Expand Up @@ -899,6 +907,59 @@
assertFileSystemAccesses("SHOW TABLES", ImmutableMultiset.of());
}


public static Stream<Arguments> partitionTransformFunctions()
{
return Stream.of(
Arguments.of("hour(d)", 4),
Arguments.of("day(d)", 2),
Arguments.of("month(d)", 2),
Arguments.of("year(d)", 2),
Arguments.of("bucket(d, 4)", 2),
Arguments.of("truncate(d, 4)", 2));
}

@ParameterizedTest
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

AFAIK the use of Parameterized tests is discouraged in trinodb/trino.
Use io.trino.plugin.iceberg.BaseIcebergConnectorTest#testDecimal as reference to change the code.

@MethodSource("partitionTransformFunctions")
public void testPartitionsArePrunedWithTemporal(String partitionTransformFunction, int expectedOccurrences)
{

Check failure on line 925 in plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergFileOperations.java

View workflow job for this annotation

GitHub Actions / test (plugin/trino-iceberg)

TestIcebergFileOperations.testPartitionsArePrunedWithTemporal(String, int)[6]

Unable to parse partitioning value: Invalid source type timestamp for transform: truncate[4]
String tableName = "test_day_transform_timestamp" + randomNameSuffix();
assertUpdate(format("CREATE TABLE %s (d TIMESTAMP(6), b BIGINT) WITH (partitioning = ARRAY['%s'])", tableName, partitionTransformFunction));

@Language("SQL") String values = "VALUES " +
"(NULL, 101)," +
"(TIMESTAMP '1969-12-25 15:13:12.876543', 8)," +
"(TIMESTAMP '1969-12-30 18:47:33.345678', 9)," +
"(TIMESTAMP '1969-12-31 00:00:00.000000', 10)," +
"(TIMESTAMP '1969-12-31 05:06:07.234567', 11)," +
"(TIMESTAMP '1970-01-01 12:03:08.456789', 12)," +
"(TIMESTAMP '2015-01-01 10:01:23.123456', 1)," +
"(TIMESTAMP '2015-01-01 11:10:02.987654', 2)," +
"(TIMESTAMP '2015-01-01 12:55:00.456789', 3)," +
"(TIMESTAMP '2015-05-15 13:05:01.234567', 4)," +
"(TIMESTAMP '2015-05-15 14:21:02.345678', 5)," +
"(TIMESTAMP '2020-02-21 15:11:11.876543', 6)," +
"(TIMESTAMP '2020-02-21 16:12:12.654321', 7)";
assertUpdate("INSERT INTO " + tableName + " " + values, 13);
assertQuery("SELECT * FROM " + tableName, values);

ImmutableMultiset<FileOperation> expectedAccesses = ImmutableMultiset.<FileOperationUtils.FileOperation>builder()
.add(new FileOperationUtils.FileOperation(SNAPSHOT, "InputFile.newStream"))
.addCopies(new FileOperationUtils.FileOperation(DATA, "InputFile.newInput"), expectedOccurrences)
.add(new FileOperationUtils.FileOperation(METADATA_JSON, "InputFile.newStream"))
.add(new FileOperationUtils.FileOperation(MANIFEST, "InputFile.newStream"))
.add(new FileOperationUtils.FileOperation(SNAPSHOT, "InputFile.length"))
.build();

assertFileSystemAccesses(
getSession(),
"SELECT * FROM " + tableName + " WHERE d >= TIMESTAMP '2015-05-15 00:00:00.000001'",
ALL_FILES,
expectedAccesses);

assertUpdate("DROP TABLE " + tableName);
}

private void assertFileSystemAccesses(@Language("SQL") String query, Multiset<FileOperation> expectedAccesses)
{
assertFileSystemAccesses(query, METADATA_FILES, expectedAccesses);
Expand Down
Loading