Skip to content

Commit

Permalink
Use Iceberg native implementation to obtain snapshot schema
Browse files Browse the repository at this point in the history
Iceberg snapshot definition in table metadata file does not have a
schema-id until Iceberg version 0.12. Time travel queries on a
table created by early versions of Trino (<=364) will fail without
this fix.
  • Loading branch information
xiacongling authored and ebyhr committed Aug 18, 2022
1 parent c48d1b1 commit adcfbe9
Show file tree
Hide file tree
Showing 6 changed files with 67 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -227,6 +227,7 @@
import static org.apache.iceberg.TableProperties.DELETE_ISOLATION_LEVEL_DEFAULT;
import static org.apache.iceberg.TableProperties.FORMAT_VERSION;
import static org.apache.iceberg.TableProperties.WRITE_LOCATION_PROVIDER_IMPL;
import static org.apache.iceberg.util.SnapshotUtil.schemaFor;

public class IcebergMetadata
implements ConnectorMetadata
Expand Down Expand Up @@ -325,7 +326,7 @@ public IcebergTableHandle getTableHandle(
long snapshotId = endVersion.map(connectorTableVersion -> getSnapshotIdFromVersion(table, connectorTableVersion))
.orElseGet(() -> resolveSnapshotId(table, name.getSnapshotId().get(), isAllowLegacySnapshotSyntax(session)));
tableSnapshotId = Optional.of(snapshotId);
tableSchema = table.schemas().get(table.snapshot(snapshotId).schemaId());
tableSchema = schemaFor(table, snapshotId);
partitionSpec = Optional.empty();
}
else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -76,6 +76,7 @@ private void configureCompatibilityTestContainer(Environment.Builder builder, Co
.withCopyFileToContainer(forHostPath(dockerFiles.getDockerFilesHostPath("conf/presto/etc/jvm.config")), containerConfigDir + "jvm.config")
.withCopyFileToContainer(forHostPath(configDir.getPath(getConfigFileFor(dockerImage))), containerConfigDir + "config.properties")
.withCopyFileToContainer(forHostPath(configDir.getPath("hive.properties")), containerConfigDir + "catalog/hive.properties")
.withCopyFileToContainer(forHostPath(configDir.getPath("iceberg.properties")), containerConfigDir + "catalog/iceberg.properties")
.withCopyFileToContainer(forHostPath(dockerFiles.getDockerFilesHostPath()), "/docker/presto-product-tests")
.withStartupCheckStrategy(new IsRunningStartupCheckStrategy())
.waitingForAll(forLogMessage(".*======== SERVER STARTED ========.*", 1), forHealthcheck())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ public List<SuiteTestRun> getTestRuns(EnvironmentConfig config)

ImmutableList<SuiteTestRun> trinoCompatibilityTestRuns = testedTrinoDockerImages().stream()
.map(image -> testOnEnvironment(EnvSinglenodeCompatibility.class, ImmutableMap.of("compatibility.testDockerImage", image))
.withGroups("configured_features", "hive_view_compatibility")
.withGroups("configured_features", "hive_view_compatibility", "iceberg_format_version_compatibility")
.build())
.collect(toImmutableList());
ImmutableList<SuiteTestRun> prestoCompatibilityTestRuns = testedPrestoDockerImages().stream()
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,2 @@
connector.name=iceberg
hive.metastore.uri=thrift://hadoop-master:9083
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,7 @@ public final class TestGroups
public static final String KAFKA = "kafka";
public static final String TWO_HIVES = "two_hives";
public static final String ICEBERG = "iceberg";
public static final String ICEBERG_FORMAT_VERSION_COMPATIBILITY = "iceberg_format_version_compatibility";
public static final String AVRO = "avro";
public static final String PHOENIX = "phoenix";
public static final String CLICKHOUSE = "clickhouse";
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,60 @@
/*
* Licensed 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 io.trino.tests.product.iceberg;

import io.trino.tempto.ProductTest;
import io.trino.tempto.assertions.QueryAssert;
import org.testng.annotations.Test;

import java.util.List;

import static com.google.common.collect.ImmutableList.toImmutableList;
import static io.trino.tempto.assertions.QueryAssert.Row.row;
import static io.trino.tempto.assertions.QueryAssert.assertThat;
import static io.trino.tests.product.TestGroups.ICEBERG_FORMAT_VERSION_COMPATIBILITY;
import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS;
import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix;
import static io.trino.tests.product.utils.QueryExecutors.onCompatibilityTestServer;
import static io.trino.tests.product.utils.QueryExecutors.onTrino;
import static java.lang.String.format;
import static org.testng.Assert.assertEquals;

public class TestIcebergFormatVersionCompatibility
extends ProductTest
{
@Test(groups = {ICEBERG_FORMAT_VERSION_COMPATIBILITY, PROFILE_SPECIFIC_TESTS})
public void testTrinoTimeTravelReadTableCreatedByEarlyVersionTrino()
{
String baseTableName = "test_trino_time_travel_read_table_created_by_early_version_trino_" + randomTableSuffix();
String tableName = format("iceberg.default.%s", baseTableName);
String snapshotsTableName = format("iceberg.default.\"%s$snapshots\"", baseTableName);

onCompatibilityTestServer().executeQuery(format("CREATE TABLE %s (c VARCHAR)", tableName));
onCompatibilityTestServer().executeQuery(format("INSERT INTO %s VALUES 'a', 'b', 'c';", tableName));

long latestSnapshotId = (long) onCompatibilityTestServer()
.executeQuery(format("SELECT snapshot_id FROM %s ORDER BY committed_at DESC FETCH FIRST 1 ROW WITH TIES", snapshotsTableName))
.getOnlyValue();
assertThat(onTrino().executeQuery(format("SELECT snapshot_id FROM %s ORDER BY committed_at DESC FETCH FIRST 1 ROW WITH TIES", snapshotsTableName)))
.containsOnly(row(latestSnapshotId));

List<QueryAssert.Row> expected = onCompatibilityTestServer().executeQuery(format("SELECT * FROM %s", tableName)).rows().stream()
.map(row -> row(row.toArray()))
.collect(toImmutableList());
assertEquals(expected.size(), 3);
assertThat(onTrino().executeQuery(format("SELECT * FROM %s FOR VERSION AS OF %d", tableName, latestSnapshotId))).containsOnly(expected);

onCompatibilityTestServer().executeQuery(format("DROP TABLE %s", tableName));
}
}

0 comments on commit adcfbe9

Please sign in to comment.