diff --git a/pom.xml b/pom.xml index f2439ec4214f..04600eb2f7ec 100644 --- a/pom.xml +++ b/pom.xml @@ -124,7 +124,7 @@ presto-benchmark-driver presto-server presto-server-rpm - presto-docs + presto-verifier presto-testing-server-launcher presto-plugin-toolkit @@ -147,6 +147,8 @@ presto-pinot presto-oracle presto-druid + presto-delta-driver + presto-delta @@ -443,7 +445,7 @@ io.prestosql.hadoop hadoop-apache - 3.2.0-9 + 3.2.0-11-delta @@ -1193,6 +1195,25 @@ ${dep.jackson.version} + + io.delta + delta-standalone_2.12 + 0.2.0 + + + + io.prestosql.delta + presto-delta-driver + shaded + ${project.version} + + + + io.prestosql + presto-delta + ${project.version} + + org.javassist diff --git a/presto-delta-driver/pom.xml b/presto-delta-driver/pom.xml new file mode 100644 index 000000000000..bb5d6e408864 --- /dev/null +++ b/presto-delta-driver/pom.xml @@ -0,0 +1,227 @@ + + + 4.0.0 + + io.prestosql + presto-root + 339 + + + + io.prestosql.delta + presto-delta-driver + presto-delta-driver + Presto - Delta Driver (Shaded Delta Standalone Reader) + jar + + + false + 2.6.7 + io.prestosql.delta.internal + + + + + io.delta + delta-standalone_2.12 + 0.2.0 + + + + org.scala-lang + scala-library + 2.12.12 + + + + com.fasterxml.jackson.core + jackson-core + ${jackson.version} + + + + com.fasterxml.jackson.core + jackson-annotations + ${jackson.version} + + + + com.fasterxml.jackson.core + jackson-databind + ${jackson.version} + + + + commons-io + commons-io + 2.8.0 + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.1 + + + + package + + shade + + + true + true + true + true + ${project.build.directory} + + + + io.delta:* + + + com.fasterxml.jackson.core:jackson-core + com.fasterxml.jackson.core:jackson-databind + com.fasterxml.jackson.core:jackson-annotations + com.fasterxml.jackson.module:jackson-module-scala_2.12 + com.fasterxml.jackson.module:jackson-module-paranamer + org.scala-lang:scala-library + commons-io:commons-io + + + com.thoughtworks.paranamer:paranamer + org.json4s:json4s-jackson_2.12 + org.json4s:json4s-core_2.12 + org.json4s:json4s-ast_2.12 + org.json4s:json4s-scalap_2.12 + org.scala-lang.modules:scala-xml_2.12 + org.apache.hadoop:hadoop-client + org.apache.hadoop:hadoop-common + com.github.mjakubowski84:parquet4s-core_2.12 + com.chuusai:shapeless_2.12 + org.typelevel:macro-compat_2.12 + org.scala-lang.modules:scala-collection-compat_2.12 + + + + io.prestosql.* + + + + + + + com.fasterxml.jackson + ${shadeBase}.com.fasterxml.jackson + + + + + com.fasterxml.jackson.module + ${shadeBase}.com.fasterxml.jackson.module + + + + + org.scala-lang + ${shadeBase}.org.scala-lang + + + + org.apache.commons.io + ${shadeBase}.org.apache.commons.io + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + META-INF/maven/** + + + + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + org.scala-lang:scala-library:jar + io.delta:delta-standalone_2.12:jar + commons-io:commons-io:jar + com.fasterxml.jackson.module:jackson-module-scala_2.12:jar + com.fasterxml.jackson.core:jackson-core:jar + com.fasterxml.jackson.core:jackson-annotations:jar + com.fasterxml.jackson.core:jackson-databind:jar + + + + + unpack + package + + unpack + + + + + io.prestosql.delta + presto-delta-driver + shaded + ${project.version} + jar + true + ${project.build.directory}/classes + **/** + + + + + + + + + + org.gaul + modernizer-maven-plugin + 1.7.1 + + true + + + + + org.apache.maven.plugins + maven-enforcer-plugin + + + + + org.alluxio:alluxio-shaded-client + org.codehaus.plexus:plexus-utils + com.fasterxml.jackson.core:jackson-core + com.google.guava:guava + com.fasterxml.jackson.core:jackson-annotations + com.fasterxml.jackson.core:jackson-databind + + + + + + + + + diff --git a/presto-delta/pom.xml b/presto-delta/pom.xml new file mode 100644 index 000000000000..06616d0d9f52 --- /dev/null +++ b/presto-delta/pom.xml @@ -0,0 +1,273 @@ + + + 4.0.0 + + io.prestosql + presto-root + 339 + + + presto-delta + Presto - Delta Table Connector + presto-plugin + + + ${project.parent.basedir} + true + + + + + io.prestosql.delta + presto-delta-driver + shaded + ${project.version} + + + io.delta + delta-standalone_2.12 + + + org.scala-lang + scala-library + + + commons-io + commons-io + + + com.fasterxml.jackson.module + jackson-module-scala_2.12 + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-databind + + + + + + io.airlift + bootstrap + + + + io.airlift + json + + + + io.airlift + log + + + + io.airlift + concurrent + + + + io.prestosql + presto-plugin-toolkit + + + + io.airlift + configuration + + + + com.google.guava + guava + + + + com.google.inject + guice + + + + javax.validation + validation-api + + + + javax.inject + javax.inject + + + + com.fasterxml.jackson.core + jackson-databind + + + + io.prestosql + presto-spi + provided + + + + io.airlift + slice + provided + + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + + io.prestosql.hive + hive-apache + + + + io.prestosql + presto-parquet + + + + io.prestosql + presto-memory-context + + + + org.weakref + jmxutils + + + + org.openjdk.jol + jol-core + provided + + + + io.prestosql + presto-hive + compile + + + io.airlift + discovery + + + io.airlift + event + + + + + + + io.prestosql + presto-tests + test + + + io.airlift + discovery + + + io.airlift + event + + + + + + io.prestosql + presto-testing + test + + + + io.prestosql.hadoop + hadoop-apache + compile + + + + io.prestosql + presto-tpch + test + + + + io.prestosql + presto-main + test-jar + test + + + + io.prestosql + presto-spi + test-jar + test + + + + io.prestosql + presto-main + test + + + + org.testng + testng + test + + + + io.airlift + testing + test + + + + io.airlift + http-server + test + + + + io.airlift + node + test + + + io.airlift + discovery + + + io.airlift + event + + + + + + javax.servlet + javax.servlet-api + test + + + + io.prestosql + presto-hive-hadoop2 + test + + + diff --git a/presto-delta/src/main/java/io/prestosql/delta/DeltaClient.java b/presto-delta/src/main/java/io/prestosql/delta/DeltaClient.java new file mode 100644 index 000000000000..c50f349ead72 --- /dev/null +++ b/presto-delta/src/main/java/io/prestosql/delta/DeltaClient.java @@ -0,0 +1,189 @@ +/* + * 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.prestosql.delta; + +import io.delta.standalone.DeltaLog; +import io.delta.standalone.Snapshot; +import io.delta.standalone.actions.AddFile; +import io.delta.standalone.actions.Metadata; +import io.prestosql.plugin.hive.HdfsEnvironment; +import io.prestosql.plugin.hive.HdfsEnvironment.HdfsContext; +import io.prestosql.spi.PrestoException; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.connector.SchemaTableName; +import io.prestosql.spi.type.TypeSignature; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import javax.inject.Inject; + +import java.io.IOException; +import java.time.Instant; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Locale; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +import static io.prestosql.delta.DeltaErrorCode.DELTA_UNSUPPORTED_DATA_FORMAT; +import static io.prestosql.delta.DeltaTypeUtils.convertDeltaDataTypePrestoDataType; +import static io.prestosql.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; +import static io.prestosql.spi.StandardErrorCode.NOT_FOUND; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +/** + * Class to interact with Delta lake table APIs. + */ +public class DeltaClient +{ + private final HdfsEnvironment hdfsEnvironment; + + @Inject + public DeltaClient(HdfsEnvironment hdfsEnvironment) + { + this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); + } + + /** + * Load the delta table. + * + * @param session Current user session + * @param schemaTableName Schema and table name referred to as in the query + * @param tableLocation Location of the Delta table on storage + * @param snapshotId Id of the snapshot to read from the Delta table + * @param snapshotAsOfTimestampMillis Latest snapshot as of given timestamp + * @return If the table is found return {@link DeltaTable}. + */ + public Optional getTable( + ConnectorSession session, + SchemaTableName schemaTableName, + String tableLocation, + Optional snapshotId, + Optional snapshotAsOfTimestampMillis) + { + Optional deltaLog = loadDeltaTableLog(session, new Path(tableLocation)); + if (!deltaLog.isPresent()) { + return Optional.empty(); + } + + // Fetch the snapshot info for given snapshot version. If no snapshot version is given, get the latest snapshot info. + // Lock the snapshot version here and use it later in the rest of the query (such as fetching file list etc.). + // If we don't lock the snapshot version here, the query may end up with schema from one version and data files from another + // version when the underlying delta table is changing while the query is running. + Snapshot snapshot; + if (snapshotId.isPresent()) { + try { + snapshot = deltaLog.get().getSnapshotForVersionAsOf(snapshotId.get()); + } + catch (IllegalArgumentException iae) { + throw new PrestoException( + NOT_FOUND, + format("Snapshot version %d does not exist in Delta table '%s'.", snapshotId.get(), schemaTableName), + iae); + } + } + else if (snapshotAsOfTimestampMillis.isPresent()) { + try { + snapshot = deltaLog.get().getSnapshotForTimestampAsOf(snapshotAsOfTimestampMillis.get()); + } + catch (IllegalArgumentException iae) { + throw new PrestoException( + NOT_FOUND, + format( + "There is no snapshot exists in Delta table '%s' that is created on or before '%s'", + schemaTableName, + Instant.ofEpochMilli(snapshotAsOfTimestampMillis.get())), + iae); + } + } + else { + snapshot = deltaLog.get().snapshot(); + } + + Metadata metadata = snapshot.getMetadata(); + String format = metadata.getFormat().getProvider(); + if (!"parquet".equalsIgnoreCase(format)) { + throw new PrestoException(DELTA_UNSUPPORTED_DATA_FORMAT, + format("Delta table %s has unsupported data format: %s. Currently only Parquet data format is supported", schemaTableName, format)); + } + + return Optional.of(new DeltaTable( + schemaTableName.getSchemaName(), + schemaTableName.getTableName(), + tableLocation, + Optional.of(snapshot.getVersion()), // lock the snapshot version + getSchema(schemaTableName, metadata))); + } + + /** + * Get the list of files corresponding to the given Delta table. + * + * @return + */ + public Iterator listFiles(ConnectorSession session, DeltaTable deltaTable) + { + Optional deltaLog = loadDeltaTableLog(session, new Path(deltaTable.getTableLocation())); + if (!deltaLog.isPresent()) { + throw new PrestoException(NOT_FOUND, + format("Delta table (%s.%s) no longer exists.", deltaTable.getSchemaName(), deltaTable.getTableName())); + } + + return deltaLog.get() + .getSnapshotForVersionAsOf(deltaTable.getSnapshotId().get()) + .getAllFiles() + .iterator(); + } + + private Optional loadDeltaTableLog(ConnectorSession session, Path tableLocation) + { + try { + HdfsContext hdfsContext = new HdfsContext(session.getIdentity()); + Configuration hdfsConf = hdfsEnvironment.getConfiguration(hdfsContext, tableLocation); + FileSystem fileSystem = hdfsEnvironment.getFileSystem(hdfsContext, tableLocation); + if (!fileSystem.isDirectory(tableLocation)) { + Optional.empty(); + } + return Optional.of(DeltaLog.forTable(hdfsConf, tableLocation)); + } + catch (IOException io) { + throw new PrestoException(GENERIC_INTERNAL_ERROR, "Failed to load Delta table: " + io.getMessage(), io); + } + } + + /** + * Utility method that returns the columns in given Delta metadata. Returned columns include regular and partition types. + * Data type from Delta is mapped to appropriate Presto data type. + */ + private List getSchema(SchemaTableName tableName, Metadata metadata) + { + Set partitionColumns = metadata.getPartitionColumns().stream() + .map(String::toLowerCase) + .collect(Collectors.toSet()); + + return Arrays.stream(metadata.getSchema().getFields()) + .map(field -> { + String columnName = field.getName().toLowerCase(Locale.US); + TypeSignature prestoType = convertDeltaDataTypePrestoDataType(tableName, columnName, field.getDataType()); + return new DeltaColumn( + columnName, + prestoType, + field.isNullable(), + partitionColumns.contains(columnName)); + }).collect(Collectors.toList()); + } +} diff --git a/presto-delta/src/main/java/io/prestosql/delta/DeltaColumn.java b/presto-delta/src/main/java/io/prestosql/delta/DeltaColumn.java new file mode 100644 index 000000000000..6c3e465d04d7 --- /dev/null +++ b/presto-delta/src/main/java/io/prestosql/delta/DeltaColumn.java @@ -0,0 +1,99 @@ +/* + * 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.prestosql.delta; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.prestosql.spi.type.TypeSignature; + +import java.util.Objects; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Strings.isNullOrEmpty; +import static java.util.Objects.requireNonNull; + +public final class DeltaColumn +{ + private final String name; + private final TypeSignature type; + private final boolean nullable; + private final boolean partition; + + @JsonCreator + public DeltaColumn( + @JsonProperty("name") String name, + @JsonProperty("type") TypeSignature type, + @JsonProperty("nullable") boolean nullable, + @JsonProperty("partition") boolean partition) + { + checkArgument(!isNullOrEmpty(name), "name is null or is empty"); + this.name = name; + this.type = requireNonNull(type, "type is null"); + this.nullable = nullable; + this.partition = partition; + } + + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty + public TypeSignature getType() + { + return type; + } + + @JsonProperty + public boolean isNullable() + { + return nullable; + } + + @JsonProperty + public boolean isPartition() + { + return partition; + } + + @Override + public int hashCode() + { + return Objects.hash(name, type, nullable, partition); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + DeltaColumn other = (DeltaColumn) obj; + return Objects.equals(this.name, other.name) && + Objects.equals(this.type, other.type) && + Objects.equals(this.nullable, other.nullable) && + Objects.equals(this.partition, other.partition); + } + + @Override + public String toString() + { + return name + ":nullable=" + nullable + ":partition=" + partition; + } +} diff --git a/presto-delta/src/main/java/io/prestosql/delta/DeltaColumnHandle.java b/presto-delta/src/main/java/io/prestosql/delta/DeltaColumnHandle.java new file mode 100644 index 000000000000..88f2b25ac780 --- /dev/null +++ b/presto-delta/src/main/java/io/prestosql/delta/DeltaColumnHandle.java @@ -0,0 +1,112 @@ +/* + * 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.prestosql.delta; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.prestosql.spi.connector.ColumnHandle; +import io.prestosql.spi.type.TypeSignature; + +import java.util.Objects; + +import static java.util.Objects.requireNonNull; + +public final class DeltaColumnHandle + implements ColumnHandle +{ + private final String connectorId; + private final String name; + private final TypeSignature dataType; + private final ColumnType columnType; + + public enum ColumnType + { + REGULAR, + PARTITION, + SUBFIELD, + } + + @JsonCreator + public DeltaColumnHandle( + @JsonProperty("connectorId") String connectorId, + @JsonProperty("columnName") String name, + @JsonProperty("dataType") TypeSignature dataType, + @JsonProperty("columnType") ColumnType columnType) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null"); + this.name = requireNonNull(name, "columnName is null"); + this.dataType = requireNonNull(dataType, "dataType is null"); + this.columnType = requireNonNull(columnType, "columnType is null"); + } + + @JsonProperty + public String getConnectorId() + { + return connectorId; + } + + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty + public TypeSignature getDataType() + { + return dataType; + } + + @JsonProperty + public ColumnType getColumnType() + { + return columnType; + } + + @Override + public String toString() + { + final StringBuilder sb = new StringBuilder("DeltaColumnHandle{"); + sb.append("connectorId='").append(connectorId).append('\''); + sb.append(", name='").append(name).append('\''); + sb.append(", dataType=").append(dataType); + sb.append(", columnType=").append(columnType); + sb.append('}'); + return sb.toString(); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + DeltaColumnHandle that = (DeltaColumnHandle) o; + return connectorId.equals(that.connectorId) && + name.equals(that.name) && + dataType.equals(that.dataType) && + columnType == that.columnType; + } + + @Override + public int hashCode() + { + return Objects.hash(connectorId, name, dataType, columnType); + } +} diff --git a/presto-delta/src/main/java/io/prestosql/delta/DeltaConfig.java b/presto-delta/src/main/java/io/prestosql/delta/DeltaConfig.java new file mode 100644 index 000000000000..7aa8131bd742 --- /dev/null +++ b/presto-delta/src/main/java/io/prestosql/delta/DeltaConfig.java @@ -0,0 +1,76 @@ +/* + * 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.prestosql.delta; + +import io.airlift.configuration.Config; + +import javax.validation.constraints.NotNull; + +public class DeltaConfig +{ + private boolean partitionPruningEnabled = true; + private boolean filterPushdownEnabled = true; + private boolean projectionPushdownEnabled = true; + private int maxSplitsBatchSize = 200; + + @NotNull + public boolean isPartitionPruningEnabled() + { + return partitionPruningEnabled; + } + + @Config("delta.partition-pruning-enabled") + public DeltaConfig setPartitionPruningEnabled(boolean partitionPruningEnabled) + { + this.partitionPruningEnabled = partitionPruningEnabled; + return this; + } + + @NotNull + public boolean isFilterPushdownEnabled() + { + return filterPushdownEnabled; + } + + @Config("delta.filter-pushdown-enabled") + public DeltaConfig setFilterPushdownEnabled(boolean filterPushdownEnabled) + { + this.filterPushdownEnabled = filterPushdownEnabled; + return this; + } + + public boolean isProjectionPushdownEnabled() + { + return projectionPushdownEnabled; + } + + @Config("delta.projection-pushdown-enabled") + public DeltaConfig setProjectionPushdownEnabled(boolean projectionPushdownEnabled) + { + this.projectionPushdownEnabled = projectionPushdownEnabled; + return this; + } + + public int getMaxSplitsBatchSize() + { + return maxSplitsBatchSize; + } + + @Config("delta.max-splits-batch-size") + public DeltaConfig setMaxSplitsBatchSize(int maxSplitsBatchSize) + { + this.maxSplitsBatchSize = maxSplitsBatchSize; + return this; + } +} diff --git a/presto-delta/src/main/java/io/prestosql/delta/DeltaConnectionHandleResolver.java b/presto-delta/src/main/java/io/prestosql/delta/DeltaConnectionHandleResolver.java new file mode 100644 index 000000000000..bea181d27992 --- /dev/null +++ b/presto-delta/src/main/java/io/prestosql/delta/DeltaConnectionHandleResolver.java @@ -0,0 +1,48 @@ +/* + * 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.prestosql.delta; + +import io.prestosql.spi.connector.ColumnHandle; +import io.prestosql.spi.connector.ConnectorHandleResolver; +import io.prestosql.spi.connector.ConnectorSplit; +import io.prestosql.spi.connector.ConnectorTableHandle; +import io.prestosql.spi.connector.ConnectorTransactionHandle; + +public class DeltaConnectionHandleResolver + implements ConnectorHandleResolver +{ + @Override + public Class getTableHandleClass() + { + return DeltaTableHandle.class; + } + + @Override + public Class getColumnHandleClass() + { + return DeltaColumnHandle.class; + } + + @Override + public Class getSplitClass() + { + return DeltaSplit.class; + } + + @Override + public Class getTransactionHandleClass() + { + return DeltaTransactionHandle.class; + } +} diff --git a/presto-delta/src/main/java/io/prestosql/delta/DeltaConnector.java b/presto-delta/src/main/java/io/prestosql/delta/DeltaConnector.java new file mode 100644 index 000000000000..0fbf1b4b40d0 --- /dev/null +++ b/presto-delta/src/main/java/io/prestosql/delta/DeltaConnector.java @@ -0,0 +1,102 @@ +/* + * 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.prestosql.delta; + +import io.airlift.bootstrap.LifeCycleManager; +import io.airlift.log.Logger; +import io.prestosql.plugin.base.classloader.ClassLoaderSafeConnectorMetadata; +import io.prestosql.plugin.base.classloader.ClassLoaderSafeConnectorPageSourceProvider; +import io.prestosql.plugin.base.classloader.ClassLoaderSafeConnectorSplitManager; +import io.prestosql.spi.connector.Connector; +import io.prestosql.spi.connector.ConnectorMetadata; +import io.prestosql.spi.connector.ConnectorPageSourceProvider; +import io.prestosql.spi.connector.ConnectorSplitManager; +import io.prestosql.spi.connector.ConnectorTransactionHandle; +import io.prestosql.spi.session.PropertyMetadata; +import io.prestosql.spi.transaction.IsolationLevel; + +import javax.inject.Inject; + +import java.util.List; + +import static io.prestosql.delta.DeltaTransactionHandle.INSTANCE; +import static java.util.Objects.requireNonNull; + +public class DeltaConnector + implements Connector +{ + private static final Logger log = Logger.get(DeltaConnector.class); + + private final LifeCycleManager lifeCycleManager; + private final DeltaMetadata metadata; + private final DeltaSplitManager splitManager; + private final DeltaSessionProperties sessionProperties; + private final DeltaPageSourceProvider pageSourceProvider; + + @Inject + public DeltaConnector( + LifeCycleManager lifeCycleManager, + DeltaMetadata metadata, + DeltaSplitManager splitManager, + DeltaSessionProperties sessionProperties, + DeltaPageSourceProvider pageSourceProvider) + { + this.lifeCycleManager = requireNonNull(lifeCycleManager, "lifeCycleManager is null"); + this.metadata = requireNonNull(metadata, "metadata is null"); + this.splitManager = requireNonNull(splitManager, "splitManager is null"); + this.sessionProperties = requireNonNull(sessionProperties, "sessionProperties is null"); + this.pageSourceProvider = requireNonNull(pageSourceProvider, "pageSourceProvider is null"); + } + + @Override + public ConnectorTransactionHandle beginTransaction(IsolationLevel isolationLevel, boolean readOnly) + { + return INSTANCE; + } + + @Override + public ConnectorMetadata getMetadata(ConnectorTransactionHandle transactionHandle) + { + return new ClassLoaderSafeConnectorMetadata(metadata, getClass().getClassLoader()); + } + + @Override + public ConnectorSplitManager getSplitManager() + { + return new ClassLoaderSafeConnectorSplitManager(splitManager, getClass().getClassLoader()); + } + + @Override + public ConnectorPageSourceProvider getPageSourceProvider() + { + return new ClassLoaderSafeConnectorPageSourceProvider(pageSourceProvider, getClass().getClassLoader()); + } + + @Override + public List> getSessionProperties() + { + return sessionProperties.getSessionProperties(); + } + + @Override + public final void shutdown() + { + try { + lifeCycleManager.stop(); + } + catch (Exception e) { + log.error(e, "Error shutting down connector"); + } + } +} diff --git a/presto-delta/src/main/java/io/prestosql/delta/DeltaConnectorFactory.java b/presto-delta/src/main/java/io/prestosql/delta/DeltaConnectorFactory.java new file mode 100644 index 000000000000..f8c5587b2435 --- /dev/null +++ b/presto-delta/src/main/java/io/prestosql/delta/DeltaConnectorFactory.java @@ -0,0 +1,87 @@ +/* + * 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.prestosql.delta; + +import com.google.inject.Injector; +import io.airlift.bootstrap.Bootstrap; +import io.airlift.json.JsonModule; +import io.prestosql.plugin.base.CatalogName; +import io.prestosql.plugin.hive.HiveHdfsModule; +import io.prestosql.plugin.hive.NodeVersion; +import io.prestosql.plugin.hive.authentication.HiveAuthenticationModule; +import io.prestosql.plugin.hive.azure.HiveAzureModule; +import io.prestosql.plugin.hive.gcs.HiveGcsModule; +import io.prestosql.plugin.hive.metastore.HiveMetastoreModule; +import io.prestosql.plugin.hive.s3.HiveS3Module; +import io.prestosql.spi.NodeManager; +import io.prestosql.spi.connector.Connector; +import io.prestosql.spi.connector.ConnectorContext; +import io.prestosql.spi.connector.ConnectorFactory; +import io.prestosql.spi.connector.ConnectorHandleResolver; + +import java.util.Map; +import java.util.Optional; + +import static com.google.common.base.Throwables.throwIfUnchecked; +import static java.util.Objects.requireNonNull; + +public class DeltaConnectorFactory + implements ConnectorFactory +{ + @Override + public String getName() + { + return "delta"; + } + + @Override + public ConnectorHandleResolver getHandleResolver() + { + return new DeltaConnectionHandleResolver(); + } + + @Override + public Connector create(String catalogName, Map requiredConfig, ConnectorContext context) + { + requireNonNull(requiredConfig, "requiredConfig is null"); + try { + Bootstrap app = new Bootstrap( + new JsonModule(), + new DeltaModule(catalogName, context.getTypeManager()), + new HiveS3Module(), + new HiveAzureModule(), + new HiveGcsModule(), + new HiveHdfsModule(), + new HiveAuthenticationModule(), + new HiveMetastoreModule(Optional.empty()), + //new CachingHiveMetastoreModule(), + binder -> { + binder.bind(NodeVersion.class).toInstance(new NodeVersion(context.getNodeManager().getCurrentNode().getVersion())); + binder.bind(NodeManager.class).toInstance(context.getNodeManager()); + binder.bind(CatalogName.class).toInstance(new CatalogName(catalogName)); + }); + + Injector injector = app + .doNotInitializeLogging() + .setRequiredConfigurationProperties(requiredConfig) + .initialize(); + + return injector.getInstance(DeltaConnector.class); + } + catch (Exception e) { + throwIfUnchecked(e); + throw new RuntimeException(e); + } + } +} diff --git a/presto-delta/src/main/java/io/prestosql/delta/DeltaConnectorId.java b/presto-delta/src/main/java/io/prestosql/delta/DeltaConnectorId.java new file mode 100644 index 000000000000..ed40d8df6462 --- /dev/null +++ b/presto-delta/src/main/java/io/prestosql/delta/DeltaConnectorId.java @@ -0,0 +1,54 @@ +/* + * 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.prestosql.delta; + +import java.util.Objects; + +import static java.util.Objects.requireNonNull; + +public final class DeltaConnectorId +{ + private final String id; + + public DeltaConnectorId(String id) + { + this.id = requireNonNull(id, "id is null"); + } + + @Override + public String toString() + { + return id; + } + + @Override + public int hashCode() + { + return Objects.hash(id); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if ((obj == null) || (getClass() != obj.getClass())) { + return false; + } + + DeltaConnectorId other = (DeltaConnectorId) obj; + return Objects.equals(this.id, other.id); + } +} diff --git a/presto-delta/src/main/java/io/prestosql/delta/DeltaErrorCode.java b/presto-delta/src/main/java/io/prestosql/delta/DeltaErrorCode.java new file mode 100644 index 000000000000..be4a14ab1ba4 --- /dev/null +++ b/presto-delta/src/main/java/io/prestosql/delta/DeltaErrorCode.java @@ -0,0 +1,48 @@ +/* + * 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.prestosql.delta; + +import io.prestosql.spi.ErrorCode; +import io.prestosql.spi.ErrorCodeSupplier; +import io.prestosql.spi.ErrorType; + +import static io.prestosql.spi.ErrorType.EXTERNAL; +import static io.prestosql.spi.ErrorType.INTERNAL_ERROR; +import static io.prestosql.spi.ErrorType.USER_ERROR; + +public enum DeltaErrorCode + implements ErrorCodeSupplier +{ + DELTA_UNSUPPORTED_COLUMN_TYPE(0, USER_ERROR), + DELTA_UNSUPPORTED_DATA_FORMAT(1, USER_ERROR), + DELTA_PARQUET_SCHEMA_MISMATCH(2, EXTERNAL), + DELTA_BAD_DATA(3, EXTERNAL), + DELTA_CANNOT_OPEN_SPLIT(4, EXTERNAL), + DELTA_MISSING_DATA(5, EXTERNAL), + DELTA_READ_DATA_ERROR(6, INTERNAL_ERROR), + DELTA_INVALID_PARTITION_VALUE(7, EXTERNAL); + + private final ErrorCode errorCode; + + DeltaErrorCode(int code, ErrorType type) + { + errorCode = new ErrorCode(code + 0x1905_0000, name(), type); + } + + @Override + public ErrorCode toErrorCode() + { + return errorCode; + } +} diff --git a/presto-delta/src/main/java/io/prestosql/delta/DeltaExpressionUtils.java b/presto-delta/src/main/java/io/prestosql/delta/DeltaExpressionUtils.java new file mode 100644 index 000000000000..788742343d64 --- /dev/null +++ b/presto-delta/src/main/java/io/prestosql/delta/DeltaExpressionUtils.java @@ -0,0 +1,243 @@ +/* + * 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.prestosql.delta; + +import com.google.common.collect.ImmutableList; +import io.airlift.slice.Slice; +import io.delta.standalone.actions.AddFile; +import io.prestosql.spi.PrestoException; +import io.prestosql.spi.connector.ColumnHandle; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.predicate.Domain; +import io.prestosql.spi.predicate.TupleDomain; +import io.prestosql.spi.predicate.ValueSet; +import io.prestosql.spi.type.StandardTypes; +import io.prestosql.spi.type.Type; +import io.prestosql.spi.type.TypeManager; + +import java.sql.Date; +import java.sql.Timestamp; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Optional; +import java.util.stream.Collectors; + +import static io.airlift.slice.Slices.utf8Slice; +import static io.prestosql.delta.DeltaColumnHandle.ColumnType.PARTITION; +import static io.prestosql.delta.DeltaErrorCode.DELTA_INVALID_PARTITION_VALUE; +import static io.prestosql.delta.DeltaErrorCode.DELTA_UNSUPPORTED_COLUMN_TYPE; +import static io.prestosql.delta.DeltaSessionProperties.isPartitionPruningEnabled; +import static java.lang.Double.doubleToRawLongBits; +import static java.lang.Double.parseDouble; +import static java.lang.Float.floatToRawIntBits; +import static java.lang.Float.parseFloat; +import static java.lang.Long.parseLong; +import static java.lang.String.format; +import static java.util.stream.Collectors.toMap; + +public class DeltaExpressionUtils +{ + private DeltaExpressionUtils() + { + } + + /** + * Split the predicate into partition and regular column predicates + */ + public static List> splitPredicate( + TupleDomain predicate) + { + Map partitionColumnPredicates = new HashMap<>(); + Map regularColumnPredicates = new HashMap<>(); + + Optional> domains = predicate.getDomains(); + if (domains.isPresent()) { + domains.get().entrySet().stream() + .forEach(domainPair -> { + DeltaColumnHandle columnHandle = (DeltaColumnHandle) domainPair.getKey(); + if (columnHandle.getColumnType() == PARTITION) { + partitionColumnPredicates.put(domainPair.getKey(), domainPair.getValue()); + } + else { + regularColumnPredicates.put(domainPair.getKey(), domainPair.getValue()); + } + }); + } + + return ImmutableList.of( + TupleDomain.withColumnDomains(partitionColumnPredicates), + TupleDomain.withColumnDomains(regularColumnPredicates)); + } + + /** + * Utility method to + * 1) remove the null value partition values. These null values cause problems later + * when used with Guava Immutable map structures. + * 2) convert column names to lower case. Presto converts the column names to lowercase. + */ + public static Map sanitizePartitionValues(Map partitionValues) + { + return partitionValues.entrySet().stream() + .filter(entry -> entry.getValue() != null) + .collect(toMap(entry -> entry.getKey().toLowerCase(Locale.ROOT), Map.Entry::getValue)); + } + + /** + * Utility method that takes an iterator of {@link AddFile}s and a predicate and returns an iterator of {@link AddFile}s + * that satisfy the predicate (predicate evaluates to a deterministic NO) + */ + public static Iterator iterateWithPredicate( + ConnectorSession session, + Iterator inputIter, + TupleDomain predicate, + TypeManager typeManager) + { + TupleDomain partitionPredicate = extractPartitionColumnsPredicate(predicate); + if (partitionPredicate.isAll() /* no partition filter */ || !isPartitionPruningEnabled(session)) { + return inputIter; + } + + if (partitionPredicate.isNone()) { + return Collections.emptyIterator(); // nothing passes the partition predicate, return empty iterator + } + + List partitionColumns = + predicate.getColumnDomains().get().stream() + .filter(entry -> entry.getColumn().getColumnType() == PARTITION) + .map(entry -> entry.getColumn()) + .collect(Collectors.toList()); + + return new Iterator() + { + private AddFile nextItem; + + @Override + public boolean hasNext() + { + if (nextItem != null) { + return true; + } + + while (inputIter.hasNext()) { + AddFile nextFile = inputIter.next(); + if (evaluatePartitionPredicate(partitionPredicate, partitionColumns, typeManager, nextFile)) { + nextItem = nextFile; + break; + } + } + + return nextItem != null; + } + + @Override + public AddFile next() + { + if (!hasNext()) { + throw new NoSuchElementException("there are no more files"); + } + AddFile toReturn = nextItem; + nextItem = null; + return toReturn; + } + }; + } + + private static TupleDomain extractPartitionColumnsPredicate(TupleDomain predicate) + { + return predicate.transform( + deltaColumnHandle -> { + if (deltaColumnHandle.getColumnType() != PARTITION) { + return null; + } + return deltaColumnHandle.getName(); + }); + } + + private static boolean evaluatePartitionPredicate( + TupleDomain partitionPredicate, + List partitionColumns, + TypeManager typeManager, + AddFile addFile) + { + Map partitionValues = sanitizePartitionValues(addFile.getPartitionValues()); + for (DeltaColumnHandle partitionColumn : partitionColumns) { + String columnName = partitionColumn.getName(); + String partitionValue = partitionValues.get(columnName.toLowerCase(Locale.ROOT)); + Domain domain = getDomain(partitionColumn, partitionValue, typeManager, addFile.getPath()); + Domain columnPredicate = partitionPredicate.getDomains().get().get(columnName); + + if (columnPredicate == null) { + continue; // there is no predicate on this column + } + + if (columnPredicate.intersect(domain).isNone()) { + return false; + } + } + + return true; + } + + private static Domain getDomain(DeltaColumnHandle columnHandle, String partitionValue, TypeManager typeManager, String filePath) + { + Type type = typeManager.getType(columnHandle.getDataType()); + if (partitionValue == null) { + return Domain.onlyNull(type); + } + + String typeBase = columnHandle.getDataType().getBase(); + try { + switch (typeBase) { + case StandardTypes.TINYINT: + case StandardTypes.SMALLINT: + case StandardTypes.INTEGER: + case StandardTypes.BIGINT: + Long intValue = parseLong(partitionValue); + return Domain.create(ValueSet.of(type, intValue), false); + case StandardTypes.REAL: + Long realValue = (long) floatToRawIntBits(parseFloat(partitionValue)); + return Domain.create(ValueSet.of(type, realValue), false); + case StandardTypes.DOUBLE: + Long doubleValue = doubleToRawLongBits(parseDouble(partitionValue)); + return Domain.create(ValueSet.of(type, doubleValue), false); + case StandardTypes.VARCHAR: + case StandardTypes.VARBINARY: + Slice sliceValue = utf8Slice(partitionValue); + return Domain.create(ValueSet.of(type, sliceValue), false); + case StandardTypes.DATE: + Long dateValue = Date.valueOf(partitionValue).toLocalDate().toEpochDay(); + return Domain.create(ValueSet.of(type, dateValue), false); + case StandardTypes.TIMESTAMP: + Long timestampValue = Timestamp.valueOf(partitionValue).getTime(); // convert to millis + return Domain.create(ValueSet.of(type, timestampValue), false); + case StandardTypes.BOOLEAN: + Boolean booleanValue = Boolean.valueOf(partitionValue); + return Domain.create(ValueSet.of(type, booleanValue), false); + default: + throw new PrestoException(DELTA_UNSUPPORTED_COLUMN_TYPE, + format("Unsupported data type '%s' for partition column %s", columnHandle.getDataType(), columnHandle.getName())); + } + } + catch (IllegalArgumentException e) { + throw new PrestoException(DELTA_INVALID_PARTITION_VALUE, + format("Can not parse partition value '%s' of type '%s' for partition column '%s' in file '%s'", + partitionValue, columnHandle.getDataType(), columnHandle.getName(), filePath)); + } + } +} diff --git a/presto-delta/src/main/java/io/prestosql/delta/DeltaMetadata.java b/presto-delta/src/main/java/io/prestosql/delta/DeltaMetadata.java new file mode 100644 index 000000000000..f79d66851abc --- /dev/null +++ b/presto-delta/src/main/java/io/prestosql/delta/DeltaMetadata.java @@ -0,0 +1,314 @@ +/* + * 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.prestosql.delta; + +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.airlift.log.Logger; +import io.prestosql.plugin.hive.authentication.HiveIdentity; +import io.prestosql.plugin.hive.metastore.HiveMetastore; +import io.prestosql.plugin.hive.metastore.Storage; +import io.prestosql.plugin.hive.metastore.Table; +import io.prestosql.spi.connector.ColumnHandle; +import io.prestosql.spi.connector.ColumnMetadata; +import io.prestosql.spi.connector.ConnectorMetadata; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.connector.ConnectorTableHandle; +import io.prestosql.spi.connector.ConnectorTableLayout; +import io.prestosql.spi.connector.ConnectorTableLayoutHandle; +import io.prestosql.spi.connector.ConnectorTableMetadata; +import io.prestosql.spi.connector.ConnectorTableProperties; +import io.prestosql.spi.connector.Constraint; +import io.prestosql.spi.connector.ConstraintApplicationResult; +import io.prestosql.spi.connector.ProjectionApplicationResult; +import io.prestosql.spi.connector.SchemaTableName; +import io.prestosql.spi.connector.SchemaTablePrefix; +import io.prestosql.spi.expression.ConnectorExpression; +import io.prestosql.spi.predicate.TupleDomain; +import io.prestosql.spi.type.TypeManager; + +import javax.inject.Inject; + +import java.util.ArrayList; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +import static com.google.common.base.Preconditions.checkArgument; +import static io.prestosql.delta.DeltaColumnHandle.ColumnType.PARTITION; +import static io.prestosql.delta.DeltaColumnHandle.ColumnType.REGULAR; +import static io.prestosql.delta.DeltaExpressionUtils.splitPredicate; +import static io.prestosql.delta.DeltaSessionProperties.isFilterPushdownEnabled; +import static io.prestosql.delta.DeltaSessionProperties.isProjectionPushdownEnabled; +import static java.util.Collections.singletonList; +import static java.util.Objects.requireNonNull; + +public class DeltaMetadata + implements ConnectorMetadata +{ + private static final Logger log = Logger.get(DeltaMetadata.class); + + /** + * Special schema used when querying a Delta table by storage location. + * Ex. SELECT * FROM delta."$PATH$"."s3://bucket/path/to/table". User is not able to list any tables + * in this schema. It is just used to query a Delta table by storage location. + */ + private static final String PATH_SCHEMA = "$PATH$"; + + private final String connectorId; + private final DeltaClient deltaClient; + private final HiveMetastore metastore; + private final TypeManager typeManager; + private final DeltaConfig config; + + @Inject + public DeltaMetadata( + DeltaConnectorId connectorId, + DeltaClient deltaClient, + HiveMetastore metastore, + TypeManager typeManager, + DeltaConfig config) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); + this.deltaClient = requireNonNull(deltaClient, "deltaClient is null"); + this.metastore = requireNonNull(metastore, "metastore is null"); + this.typeManager = requireNonNull(typeManager, "typeManager is null"); + this.config = requireNonNull(config, "config is null"); + } + + @Override + public List listSchemaNames(ConnectorSession session) + { + ArrayList schemas = new ArrayList<>(); + schemas.addAll(metastore.getAllDatabases()); + schemas.add(PATH_SCHEMA.toLowerCase(Locale.ROOT)); + return schemas; + } + + @Override + public DeltaTableHandle getTableHandle(ConnectorSession session, SchemaTableName schemaTableName) + { + String schemaName = schemaTableName.getSchemaName(); + String tableName = schemaTableName.getTableName(); + if (!listSchemaNames(session).contains(schemaName)) { + return null; // indicates table doesn't exist + } + + DeltaTableName deltaTableName = DeltaTableName.from(tableName); + String tableLocation; + if (PATH_SCHEMA.equalsIgnoreCase(schemaName)) { + tableLocation = deltaTableName.getTableNameOrPath(); + } + else { + Optional metastoreTable = metastore.getTable( + new HiveIdentity(session), + schemaName, + deltaTableName.getTableNameOrPath()); + if (!metastoreTable.isPresent()) { + return null; // indicates table doesn't exist + } + + Map tableParameters = metastoreTable.get().getParameters(); + Storage storage = metastoreTable.get().getStorage(); + tableLocation = storage.getLocation(); + + // Delta table written using Spark and Hive have set the table parameter + // "spark.sql.sources.provider = delta". If this property is found table + // location is found in SerDe properties with key "path". + if ("delta".equalsIgnoreCase(tableParameters.get("spark.sql.sources.provider"))) { + tableLocation = storage.getSerdeParameters().get("path"); + if (Strings.isNullOrEmpty(tableLocation)) { + log.warn("Location key ('path') is missing in SerDe properties for table %s. " + + "Using the 'location' attribute as the table location.", schemaTableName); + // fallback to using the location attribute + tableLocation = storage.getLocation(); + } + } + } + + Optional table = deltaClient.getTable( + session, + schemaTableName, + tableLocation, + deltaTableName.getSnapshotId(), + deltaTableName.getSnapshotAsOfTimestamp()); + if (table.isPresent()) { + return new DeltaTableHandle(connectorId, table.get(), TupleDomain.all(), Optional.empty()); + } + return null; + } + + @Override + public boolean usesLegacyTableLayouts() + { + return false; + } + + @Override + public ConnectorTableProperties getTableProperties(ConnectorSession session, ConnectorTableHandle tableHandle) + { + return new ConnectorTableProperties(); + } + + @Override + public Optional> applyFilter(ConnectorSession session, ConnectorTableHandle handle, Constraint constraint) + { + if (!isFilterPushdownEnabled(session)) { + return Optional.empty(); + } + + DeltaTableHandle table = (DeltaTableHandle) handle; + if (constraint.getSummary().equals(table.getPredicate())) { + return Optional.empty(); + } + + // Split the predicate into partition column predicate and other column predicates + // Only the partition column predicate is fully enforced. Other predicate is partially enforced (best effort). + List> predicate = splitPredicate(constraint.getSummary()); + TupleDomain enforcedPredicate = predicate.get(0); + TupleDomain unenforcedPredicate = predicate.get(1); + + DeltaTableHandle newDeltaTableHandle = table.withPredicate( + constraint.getSummary().transform(DeltaColumnHandle.class::cast), + constraint.getSummary().toString(session)); + + return Optional.of(new ConstraintApplicationResult<>( + newDeltaTableHandle, + unenforcedPredicate)); + } + + @Override + public Optional> applyProjection( + ConnectorSession session, ConnectorTableHandle handle, + List projections, + Map assignments) + { + if (!isProjectionPushdownEnabled(session)) { + return Optional.empty(); + } + + // TODO: implement + return Optional.empty(); + } + + @Override + public ConnectorTableLayout getTableLayout(ConnectorSession session, ConnectorTableLayoutHandle handle) + { + return new ConnectorTableLayout(handle); + } + + @Override + public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle table) + { + DeltaTableHandle deltaTableHandle = (DeltaTableHandle) table; + checkConnectorId(deltaTableHandle); + return getTableMetadata(session, deltaTableHandle.toSchemaTableName()); + } + + @Override + public List listTables(ConnectorSession session, Optional schemaName) + { + List schemaNames = schemaName.isEmpty() ? listSchemaNames(session) : ImmutableList.of(schemaName.get()); + + ImmutableList.Builder tableNames = ImmutableList.builder(); + for (String schema : schemaNames) { + for (String table : metastore.getAllTables(schema)) { + tableNames.add(new SchemaTableName(schema, table)); + } + } + return tableNames.build(); + } + + @Override + public Map getColumnHandles(ConnectorSession session, ConnectorTableHandle tableHandle) + { + DeltaTableHandle deltaTableHandle = (DeltaTableHandle) tableHandle; + checkConnectorId(deltaTableHandle); + + ImmutableMap.Builder columnHandles = ImmutableMap.builder(); + for (DeltaColumn column : deltaTableHandle.getDeltaTable().getColumns()) { + columnHandles.put( + column.getName(), + new DeltaColumnHandle( + connectorId, + column.getName(), + column.getType(), + column.isPartition() ? PARTITION : REGULAR)); + } + return columnHandles.build(); + } + + @Override + public Map> listTableColumns(ConnectorSession session, SchemaTablePrefix prefix) + { + requireNonNull(prefix, "prefix is null"); + ImmutableMap.Builder> columns = ImmutableMap.builder(); + for (SchemaTableName tableName : listTables(session, prefix)) { + ConnectorTableMetadata tableMetadata = getTableMetadata(session, tableName); + // table can disappear during listing operation + if (tableMetadata != null) { + columns.put(tableName, tableMetadata.getColumns()); + } + } + return columns.build(); + } + + private ConnectorTableMetadata getTableMetadata(ConnectorSession session, SchemaTableName tableName) + { + DeltaTableHandle tableHandle = getTableHandle(session, tableName); + if (tableHandle == null) { + return null; + } + + List columnMetadata = tableHandle.getDeltaTable().getColumns().stream() + .map(column -> getColumnMetadata(column)) + .collect(Collectors.toList()); + + return new ConnectorTableMetadata(tableName, columnMetadata); + } + + @Override + public ColumnMetadata getColumnMetadata(ConnectorSession session, ConnectorTableHandle tableHandle, ColumnHandle columnHandle) + { + return getColumnMetadata(columnHandle); + } + + private ColumnMetadata getColumnMetadata(ColumnHandle columnHandle) + { + DeltaColumnHandle deltaColumnHandle = (DeltaColumnHandle) columnHandle; + return new ColumnMetadata(deltaColumnHandle.getName(), typeManager.getType(deltaColumnHandle.getDataType())); + } + + private List listTables(ConnectorSession session, SchemaTablePrefix prefix) + { + if (prefix.getTable().isEmpty()) { + return listTables(session, prefix.getSchema()); + } + + return singletonList(new SchemaTableName(prefix.getSchema().get(), prefix.getTable().get())); + } + + private ColumnMetadata getColumnMetadata(DeltaColumn deltaColumn) + { + return new ColumnMetadata(deltaColumn.getName(), typeManager.getType(deltaColumn.getType())); + } + + private void checkConnectorId(DeltaTableHandle tableHandle) + { + checkArgument(tableHandle.getConnectorId().equals(connectorId), "table handle is not for this connector"); + } +} diff --git a/presto-delta/src/main/java/io/prestosql/delta/DeltaModule.java b/presto-delta/src/main/java/io/prestosql/delta/DeltaModule.java new file mode 100644 index 000000000000..3eb4c20abf86 --- /dev/null +++ b/presto-delta/src/main/java/io/prestosql/delta/DeltaModule.java @@ -0,0 +1,117 @@ +/* + * 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.prestosql.delta; + +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.deser.std.FromStringDeserializer; +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.google.inject.Scopes; +import io.prestosql.plugin.base.CatalogName; +import io.prestosql.plugin.hive.DynamicConfigurationProvider; +import io.prestosql.plugin.hive.FileFormatDataSourceStats; +import io.prestosql.plugin.hive.HdfsConfiguration; +import io.prestosql.plugin.hive.HdfsConfigurationInitializer; +import io.prestosql.plugin.hive.HdfsEnvironment; +import io.prestosql.plugin.hive.HiveConfig; +import io.prestosql.plugin.hive.HiveHdfsConfiguration; +import io.prestosql.plugin.hive.metastore.MetastoreConfig; +import io.prestosql.spi.type.Type; +import io.prestosql.spi.type.TypeId; +import io.prestosql.spi.type.TypeManager; + +import javax.inject.Inject; +import javax.inject.Singleton; + +import java.util.concurrent.ExecutorService; + +import static com.google.inject.multibindings.Multibinder.newSetBinder; +import static io.airlift.concurrent.Threads.daemonThreadsNamed; +import static io.airlift.configuration.ConfigBinder.configBinder; +import static io.airlift.json.JsonBinder.jsonBinder; +import static io.airlift.json.JsonCodec.listJsonCodec; +import static io.airlift.json.JsonCodecBinder.jsonCodecBinder; +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.Executors.newCachedThreadPool; +import static org.weakref.jmx.ObjectNames.generatedNameOf; +import static org.weakref.jmx.guice.ExportBinder.newExporter; + +public class DeltaModule + implements Module +{ + private final String connectorId; + private final TypeManager typeManager; + + public DeltaModule(String connectorId, TypeManager typeManager) + { + this.connectorId = requireNonNull(connectorId, "connector id is null"); + this.typeManager = requireNonNull(typeManager, "typeManager is null"); + } + + @Override + public void configure(Binder binder) + { + binder.bind(TypeManager.class).toInstance(typeManager); + + binder.bind(DeltaConnector.class).in(Scopes.SINGLETON); + binder.bind(DeltaConnectorId.class).toInstance(new DeltaConnectorId(connectorId)); + binder.bind(DeltaMetadata.class).in(Scopes.SINGLETON); + binder.bind(DeltaClient.class).in(Scopes.SINGLETON); + binder.bind(DeltaSplitManager.class).in(Scopes.SINGLETON); + binder.bind(DeltaPageSourceProvider.class).in(Scopes.SINGLETON); + binder.bind(DeltaSessionProperties.class).in(Scopes.SINGLETON); + configBinder(binder).bindConfig(DeltaConfig.class); + + configBinder(binder).bindConfig(MetastoreConfig.class); + configBinder(binder).bindConfig(HiveConfig.class); + + binder.bind(HdfsConfigurationInitializer.class).in(Scopes.SINGLETON); + binder.bind(HdfsConfiguration.class).to(HiveHdfsConfiguration.class).in(Scopes.SINGLETON); + newSetBinder(binder, DynamicConfigurationProvider.class); + binder.bind(HdfsEnvironment.class).in(Scopes.SINGLETON); + + jsonBinder(binder).addDeserializerBinding(Type.class).to(TypeDeserializer.class); + jsonCodecBinder(binder).bindMapJsonCodec(String.class, listJsonCodec(DeltaTable.class)); + + binder.bind(FileFormatDataSourceStats.class).in(Scopes.SINGLETON); + newExporter(binder).export(FileFormatDataSourceStats.class).as(generatedNameOf(FileFormatDataSourceStats.class, connectorId)); + } + + public static final class TypeDeserializer + extends FromStringDeserializer + { + private final TypeManager typeManager; + + @Inject + public TypeDeserializer(TypeManager typeManager) + { + super(Type.class); + this.typeManager = requireNonNull(typeManager, "metadata is null"); + } + + @Override + protected Type _deserialize(String value, DeserializationContext context) + { + return typeManager.getType(TypeId.of(value)); + } + } + + @Singleton + @Provides + public ExecutorService createDeltaClientExecutor(CatalogName catalogName) + { + return newCachedThreadPool(daemonThreadsNamed("delta-" + catalogName + "-%s")); + } +} diff --git a/presto-delta/src/main/java/io/prestosql/delta/DeltaPageSource.java b/presto-delta/src/main/java/io/prestosql/delta/DeltaPageSource.java new file mode 100644 index 000000000000..dcf1f7f3add8 --- /dev/null +++ b/presto-delta/src/main/java/io/prestosql/delta/DeltaPageSource.java @@ -0,0 +1,145 @@ +/* + * 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.prestosql.delta; + +import io.prestosql.spi.Page; +import io.prestosql.spi.PrestoException; +import io.prestosql.spi.block.Block; +import io.prestosql.spi.block.RunLengthEncodedBlock; +import io.prestosql.spi.connector.ConnectorPageSource; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +import static io.prestosql.delta.DeltaColumnHandle.ColumnType.PARTITION; +import static io.prestosql.delta.DeltaErrorCode.DELTA_READ_DATA_ERROR; +import static java.util.Objects.requireNonNull; + +/** + * {@link ConnectorPageSource} implementation for Delta tables that prefills + * partition column blocks and combines them with regular column blocks returned + * by the underlying file reader {@link ConnectorPageSource} implementation. + */ +public class DeltaPageSource + implements ConnectorPageSource +{ + private final List columnHandles; + private final ConnectorPageSource dataPageSource; + private final Map partitionValues; + + /** + * Create a DeltaPageSource + * + * @param columnHandles List of columns (includes partition and regular) in order for which data needed in output. + * @param partitionValues Partition values (partition column -> partition value map). + * @param dataPageSource Initialized underlying file reader which returns the data for regular columns. + */ + public DeltaPageSource( + List columnHandles, + Map partitionValues, + ConnectorPageSource dataPageSource) + { + this.columnHandles = requireNonNull(columnHandles, "columnHandles is null"); + this.dataPageSource = requireNonNull(dataPageSource, "dataPageSource is null"); + this.partitionValues = requireNonNull(partitionValues, "partitionValues is null"); + } + + @Override + public Page getNextPage() + { + try { + Page dataPage = dataPageSource.getNextPage(); + if (dataPage == null) { + return null; // reader is done + } + int positionCount = dataPage.getPositionCount(); + + int dataColumnIndex = 0; + int columnIndex = 0; + Block[] blocksWithPartitionColumns = new Block[columnHandles.size()]; + for (DeltaColumnHandle columnHandle : columnHandles) { + if (columnHandle.getColumnType() == PARTITION) { + Block partitionValue = partitionValues.get(columnHandle.getName()); + blocksWithPartitionColumns[columnIndex++] = new RunLengthEncodedBlock(partitionValue, positionCount); + } + else { + blocksWithPartitionColumns[columnIndex++] = (dataPage.getBlock(dataColumnIndex)); + dataColumnIndex++; + } + } + return new Page(positionCount, blocksWithPartitionColumns); + } + catch (PrestoException e) { + closeWithSuppression(e); + throw e; // already properly handled exception - throw without any additional info + } + catch (RuntimeException e) { + closeWithSuppression(e); + throw new PrestoException(DELTA_READ_DATA_ERROR, e); + } + } + + private void closeWithSuppression(Throwable throwable) + { + requireNonNull(throwable, "throwable is null"); + try { + close(); + } + catch (Exception e) { + // Self-suppression not permitted + if (e != throwable) { + throwable.addSuppressed(e); + } + } + } + + @Override + public long getCompletedBytes() + { + return dataPageSource.getCompletedBytes(); + } + + @Override + public long getReadTimeNanos() + { + return dataPageSource.getReadTimeNanos(); + } + + @Override + public boolean isFinished() + { + return dataPageSource.isFinished(); + } + + @Override + public long getSystemMemoryUsage() + { + return dataPageSource.getSystemMemoryUsage(); + } + + @Override + public void close() + throws IOException + { + dataPageSource.close(); + } + + @Override + public CompletableFuture isBlocked() + { + return dataPageSource.isBlocked(); + } +} diff --git a/presto-delta/src/main/java/io/prestosql/delta/DeltaPageSourceProvider.java b/presto-delta/src/main/java/io/prestosql/delta/DeltaPageSourceProvider.java new file mode 100644 index 000000000000..250e6a7acd4e --- /dev/null +++ b/presto-delta/src/main/java/io/prestosql/delta/DeltaPageSourceProvider.java @@ -0,0 +1,318 @@ +/* + * 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.prestosql.delta; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.prestosql.memory.context.AggregatedMemoryContext; +import io.prestosql.parquet.Field; +import io.prestosql.parquet.ParquetCorruptionException; +import io.prestosql.parquet.ParquetDataSource; +import io.prestosql.parquet.ParquetReaderOptions; +import io.prestosql.parquet.RichColumnDescriptor; +import io.prestosql.parquet.predicate.Predicate; +import io.prestosql.parquet.reader.MetadataReader; +import io.prestosql.parquet.reader.ParquetReader; +import io.prestosql.plugin.hive.FileFormatDataSourceStats; +import io.prestosql.plugin.hive.HdfsEnvironment; +import io.prestosql.plugin.hive.HdfsEnvironment.HdfsContext; +import io.prestosql.plugin.hive.parquet.ParquetPageSource; +import io.prestosql.spi.PrestoException; +import io.prestosql.spi.block.Block; +import io.prestosql.spi.connector.ColumnHandle; +import io.prestosql.spi.connector.ConnectorPageSource; +import io.prestosql.spi.connector.ConnectorPageSourceProvider; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.connector.ConnectorSplit; +import io.prestosql.spi.connector.ConnectorTableHandle; +import io.prestosql.spi.connector.ConnectorTransactionHandle; +import io.prestosql.spi.predicate.Domain; +import io.prestosql.spi.predicate.TupleDomain; +import io.prestosql.spi.predicate.Utils; +import io.prestosql.spi.type.Type; +import io.prestosql.spi.type.TypeManager; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.security.AccessControlException; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.FileMetaData; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.io.MessageColumnIO; +import org.apache.parquet.schema.MessageType; + +import javax.inject.Inject; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Strings.nullToEmpty; +import static io.prestosql.delta.DeltaColumnHandle.ColumnType.PARTITION; +import static io.prestosql.delta.DeltaColumnHandle.ColumnType.REGULAR; +import static io.prestosql.delta.DeltaColumnHandle.ColumnType.SUBFIELD; +import static io.prestosql.delta.DeltaErrorCode.DELTA_BAD_DATA; +import static io.prestosql.delta.DeltaErrorCode.DELTA_CANNOT_OPEN_SPLIT; +import static io.prestosql.delta.DeltaErrorCode.DELTA_MISSING_DATA; +import static io.prestosql.delta.DeltaTypeUtils.convertPartitionValue; +import static io.prestosql.memory.context.AggregatedMemoryContext.newSimpleAggregatedMemoryContext; +import static io.prestosql.parquet.ParquetTypeUtils.getColumnIO; +import static io.prestosql.parquet.ParquetTypeUtils.getDescriptors; +import static io.prestosql.parquet.ParquetTypeUtils.getParquetTypeByName; +import static io.prestosql.parquet.ParquetTypeUtils.lookupColumnByName; +import static io.prestosql.parquet.predicate.PredicateUtils.buildPredicate; +import static io.prestosql.parquet.predicate.PredicateUtils.predicateMatches; +import static io.prestosql.plugin.hive.parquet.HdfsParquetDataSource.buildHdfsParquetDataSource; +import static io.prestosql.plugin.hive.parquet.ParquetColumnIOConverter.constructField; +import static io.prestosql.spi.StandardErrorCode.PERMISSION_DENIED; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.toMap; + +public class DeltaPageSourceProvider + implements ConnectorPageSourceProvider +{ + private final HdfsEnvironment hdfsEnvironment; + private final TypeManager typeManager; + private final FileFormatDataSourceStats fileFormatDataSourceStats; + + @Inject + public DeltaPageSourceProvider( + HdfsEnvironment hdfsEnvironment, + TypeManager typeManager, + FileFormatDataSourceStats fileFormatDataSourceStats) + { + this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); + this.typeManager = requireNonNull(typeManager, "typeManager is null"); + this.fileFormatDataSourceStats = requireNonNull(fileFormatDataSourceStats, "fileFormatDataSourceStats is null"); + } + + @Override + public ConnectorPageSource createPageSource( + ConnectorTransactionHandle transaction, + ConnectorSession session, + ConnectorSplit split, + ConnectorTableHandle table, + List columns, + TupleDomain dynamicFilter) + { + DeltaSplit deltaSplit = (DeltaSplit) split; + DeltaTableHandle deltaTableHandle = (DeltaTableHandle) table; + + HdfsContext hdfsContext = new HdfsContext(session.getIdentity()); + Path filePath = new Path(deltaSplit.getFilePath()); + List deltaColumnHandles = columns.stream() + .map(DeltaColumnHandle.class::cast) + .collect(Collectors.toList()); + + List regularColumnHandles = deltaColumnHandles.stream() + .filter(columnHandle -> columnHandle.getColumnType() != PARTITION) + .collect(Collectors.toList()); + + ConnectorPageSource dataPageSource = createParquetPageSource( + hdfsEnvironment, + session.getUser(), + hdfsEnvironment.getConfiguration(hdfsContext, filePath), + filePath, + deltaSplit.getStart(), + deltaSplit.getLength(), + deltaSplit.getFileSize(), + regularColumnHandles, + typeManager, + deltaTableHandle.getPredicate(), + fileFormatDataSourceStats); + + return new DeltaPageSource( + deltaColumnHandles, + convertPartitionValues(deltaColumnHandles, deltaSplit.getPartitionValues()), + dataPageSource); + } + + /** + * Go through all the output columns, identify the partition columns and convert the partition values to Presto internal format. + */ + private Map convertPartitionValues( + List allColumns, + Map partitionValues) + { + return allColumns.stream() + .filter(columnHandle -> columnHandle.getColumnType() == PARTITION) + .collect(toMap( + DeltaColumnHandle::getName, + columnHandle -> + Utils.nativeValueToBlock( + typeManager.getType(columnHandle.getDataType()), + convertPartitionValue( + columnHandle.getName(), + partitionValues.get(columnHandle.getName()), + columnHandle.getDataType())))); + } + + private static ConnectorPageSource createParquetPageSource( + HdfsEnvironment hdfsEnvironment, + String user, + Configuration configuration, + Path path, + long start, + long length, + long fileSize, + List columns, + TypeManager typeManager, + TupleDomain effectivePredicate, + FileFormatDataSourceStats stats) + { + AggregatedMemoryContext systemMemoryContext = newSimpleAggregatedMemoryContext(); + + ParquetDataSource dataSource = null; + try { + FSDataInputStream inputStream = hdfsEnvironment + .getFileSystem(user, path, configuration) + .open(path); + ParquetReaderOptions readerOptions = new ParquetReaderOptions(); + dataSource = buildHdfsParquetDataSource(inputStream, path, fileSize, stats, readerOptions); + ParquetMetadata parquetMetadata = MetadataReader.readFooter(inputStream, path, fileSize); + + FileMetaData fileMetaData = parquetMetadata.getFileMetaData(); + MessageType fileSchema = fileMetaData.getSchema(); + + Optional message = columns.stream() + .filter(column -> column.getColumnType() == REGULAR) + .map(column -> getParquetType(fileSchema, column)) + .filter(Optional::isPresent) + .map(Optional::get) + .map(type -> new MessageType(fileSchema.getName(), type)) + .reduce(MessageType::union); + + MessageType requestedSchema = message + .orElse(new MessageType(fileSchema.getName(), ImmutableList.of())); + + ImmutableList.Builder footerBlocks = ImmutableList.builder(); + for (BlockMetaData block : parquetMetadata.getBlocks()) { + long firstDataPage = block.getColumns().get(0).getFirstDataPageOffset(); + if (firstDataPage >= start && firstDataPage < start + length) { + footerBlocks.add(block); + } + } + + Map, RichColumnDescriptor> descriptorsByPath = getDescriptors(fileSchema, requestedSchema); + TupleDomain parquetTupleDomain = getParquetTupleDomain(descriptorsByPath, effectivePredicate); + Predicate parquetPredicate = buildPredicate(requestedSchema, parquetTupleDomain, descriptorsByPath); + final ParquetDataSource finalDataSource = dataSource; + ImmutableList.Builder blocks = ImmutableList.builder(); + for (BlockMetaData block : footerBlocks.build()) { + if (predicateMatches( + parquetPredicate, + block, + finalDataSource, + descriptorsByPath, + parquetTupleDomain, + readerOptions.isFailOnCorruptedStatistics())) { + blocks.add(block); + } + } + MessageColumnIO messageColumnIO = getColumnIO(fileSchema, requestedSchema); + ParquetReader parquetReader = new ParquetReader( + Optional.ofNullable(fileMetaData.getCreatedBy()), + messageColumnIO, + blocks.build(), + dataSource, + systemMemoryContext, + readerOptions); + + ImmutableList.Builder namesBuilder = ImmutableList.builder(); + ImmutableList.Builder typesBuilder = ImmutableList.builder(); + ImmutableList.Builder> fieldsBuilder = ImmutableList.builder(); + for (DeltaColumnHandle column : columns) { + checkArgument(column.getColumnType() == REGULAR || column.getColumnType() == SUBFIELD, + "column type must be regular or subfield column"); + + String name = column.getName(); + Type type = typeManager.getType(column.getDataType()); + + namesBuilder.add(name); + typesBuilder.add(type); + + if (getParquetType(fileSchema, column).isPresent()) { + String columnName = name; + fieldsBuilder.add(constructField(type, lookupColumnByName(messageColumnIO, columnName))); + } + else { + fieldsBuilder.add(Optional.empty()); + } + } + return new ParquetPageSource( + parquetReader, + typesBuilder.build(), + fieldsBuilder.build()); + } + catch (Exception e) { + try { + if (dataSource != null) { + dataSource.close(); + } + } + catch (IOException ignored) { + } + if (e instanceof PrestoException) { + throw (PrestoException) e; + } + if (e instanceof ParquetCorruptionException) { + throw new PrestoException(DELTA_BAD_DATA, e); + } + if (e instanceof AccessControlException) { + throw new PrestoException(PERMISSION_DENIED, e.getMessage(), e); + } + if (nullToEmpty(e.getMessage()).trim().equals("Filesystem closed") || e instanceof FileNotFoundException) { + throw new PrestoException(DELTA_CANNOT_OPEN_SPLIT, e); + } + String message = format("Error opening Hive split %s (offset=%s, length=%s): %s", path, start, length, e.getMessage()); + if (e.getClass().getSimpleName().equals("BlockMissingException")) { + throw new PrestoException(DELTA_MISSING_DATA, message, e); + } + throw new PrestoException(DELTA_CANNOT_OPEN_SPLIT, message, e); + } + } + + public static TupleDomain getParquetTupleDomain(Map, RichColumnDescriptor> descriptorsByPath, TupleDomain effectivePredicate) + { + if (effectivePredicate.isNone()) { + return TupleDomain.none(); + } + + ImmutableMap.Builder predicate = ImmutableMap.builder(); + for (Map.Entry entry : effectivePredicate.getDomains().get().entrySet()) { + DeltaColumnHandle columnHandle = entry.getKey(); + + RichColumnDescriptor descriptor = descriptorsByPath.get(ImmutableList.of(columnHandle.getName())); + if (descriptor != null) { + predicate.put(descriptor, entry.getValue()); + } + } + return TupleDomain.withColumnDomains(predicate.build()); + } + + public static Optional getParquetType( + MessageType messageType, + DeltaColumnHandle column) + { + org.apache.parquet.schema.Type type = getParquetTypeByName(column.getName(), messageType); + return Optional.of(type); + } +} diff --git a/presto-delta/src/main/java/io/prestosql/delta/DeltaPlugin.java b/presto-delta/src/main/java/io/prestosql/delta/DeltaPlugin.java new file mode 100644 index 000000000000..352618cf4399 --- /dev/null +++ b/presto-delta/src/main/java/io/prestosql/delta/DeltaPlugin.java @@ -0,0 +1,28 @@ +/* + * 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.prestosql.delta; + +import com.google.common.collect.ImmutableList; +import io.prestosql.spi.Plugin; +import io.prestosql.spi.connector.ConnectorFactory; + +public class DeltaPlugin + implements Plugin +{ + @Override + public Iterable getConnectorFactories() + { + return ImmutableList.of(new DeltaConnectorFactory()); + } +} diff --git a/presto-delta/src/main/java/io/prestosql/delta/DeltaSessionProperties.java b/presto-delta/src/main/java/io/prestosql/delta/DeltaSessionProperties.java new file mode 100644 index 000000000000..b7d744114c86 --- /dev/null +++ b/presto-delta/src/main/java/io/prestosql/delta/DeltaSessionProperties.java @@ -0,0 +1,74 @@ +/* + * 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.prestosql.delta; + +import com.google.common.collect.ImmutableList; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.session.PropertyMetadata; + +import javax.inject.Inject; + +import java.util.List; + +import static io.prestosql.spi.session.PropertyMetadata.booleanProperty; + +public final class DeltaSessionProperties +{ + public static final String PARTITION_PRUNING_ENABLED = "partition_pruning_enabled"; + public static final String FILTER_PUSHDOWN_ENABLED = "filter_pushdown_enabled"; + public static final String PROJECTION_PUSHDOWN_ENABLED = "projection_pushdown_enabled"; + + private final List> sessionProperties; + + @Inject + public DeltaSessionProperties(DeltaConfig deltaConfig) + { + sessionProperties = ImmutableList.of( + booleanProperty( + PARTITION_PRUNING_ENABLED, + "Enable partition pruning in Delta tables", + deltaConfig.isPartitionPruningEnabled(), + false), + booleanProperty( + FILTER_PUSHDOWN_ENABLED, + "Enable filter pushdown into Delta tables", + deltaConfig.isFilterPushdownEnabled(), + false), + booleanProperty( + PROJECTION_PUSHDOWN_ENABLED, + "Enable project pushdown into Delta tables", + deltaConfig.isProjectionPushdownEnabled(), + false)); + } + + public List> getSessionProperties() + { + return sessionProperties; + } + + public static boolean isPartitionPruningEnabled(ConnectorSession session) + { + return session.getProperty(PARTITION_PRUNING_ENABLED, Boolean.class); + } + + public static boolean isFilterPushdownEnabled(ConnectorSession session) + { + return session.getProperty(FILTER_PUSHDOWN_ENABLED, Boolean.class); + } + + public static boolean isProjectionPushdownEnabled(ConnectorSession session) + { + return session.getProperty(PROJECTION_PUSHDOWN_ENABLED, Boolean.class); + } +} diff --git a/presto-delta/src/main/java/io/prestosql/delta/DeltaSplit.java b/presto-delta/src/main/java/io/prestosql/delta/DeltaSplit.java new file mode 100644 index 000000000000..8f6ba34066b4 --- /dev/null +++ b/presto-delta/src/main/java/io/prestosql/delta/DeltaSplit.java @@ -0,0 +1,131 @@ +/* + * 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.prestosql.delta; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.prestosql.spi.HostAddress; +import io.prestosql.spi.connector.ConnectorSplit; + +import java.util.List; +import java.util.Map; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +public class DeltaSplit + implements ConnectorSplit +{ + private final String connectorId; + private final String schema; + private final String table; + private final String filePath; + private final long start; + private final long length; + private final long fileSize; + private final Map partitionValues; + + @JsonCreator + public DeltaSplit( + @JsonProperty("connectorId") String connectorId, + @JsonProperty("schemaName") String schema, + @JsonProperty("tableName") String table, + @JsonProperty("filePath") String filePath, + @JsonProperty("start") long start, + @JsonProperty("length") long length, + @JsonProperty("fileSize") long fileSize, + @JsonProperty("partitionValues") Map partitionValues) + { + checkArgument(start >= 0, "start must be positive"); + checkArgument(length >= 0, "length must be positive"); + checkArgument(fileSize >= 0, "fileSize must be positive"); + + this.connectorId = requireNonNull(connectorId, "connector id is null"); + this.schema = requireNonNull(schema, "schema name is null"); + this.table = requireNonNull(table, "table name is null"); + this.filePath = requireNonNull(filePath, "filePath name is null"); + this.start = start; + this.length = length; + this.fileSize = fileSize; + this.partitionValues = ImmutableMap.copyOf(requireNonNull(partitionValues, "partitionValues id is null")); + } + + @JsonProperty + public String getConnectorId() + { + return connectorId; + } + + @JsonProperty + public String getSchema() + { + return schema; + } + + @JsonProperty + public String getTable() + { + return table; + } + + @JsonProperty + public String getFilePath() + { + return filePath; + } + + @JsonProperty + public long getStart() + { + return start; + } + + @JsonProperty + public long getLength() + { + return length; + } + + @JsonProperty + public long getFileSize() + { + return fileSize; + } + + @JsonProperty + public Map getPartitionValues() + { + return partitionValues; + } + + @Override + public boolean isRemotelyAccessible() + { + return true; + } + + @Override + public List getAddresses() + { + return ImmutableList.of(); + } + + @Override + public Object getInfo() + { + return this; + } +} diff --git a/presto-delta/src/main/java/io/prestosql/delta/DeltaSplitManager.java b/presto-delta/src/main/java/io/prestosql/delta/DeltaSplitManager.java new file mode 100644 index 000000000000..db71c0140114 --- /dev/null +++ b/presto-delta/src/main/java/io/prestosql/delta/DeltaSplitManager.java @@ -0,0 +1,121 @@ +/* + * 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.prestosql.delta; + +import io.delta.standalone.actions.AddFile; +import io.prestosql.spi.connector.ConnectorPartitionHandle; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.connector.ConnectorSplit; +import io.prestosql.spi.connector.ConnectorSplitManager; +import io.prestosql.spi.connector.ConnectorSplitSource; +import io.prestosql.spi.connector.ConnectorTableHandle; +import io.prestosql.spi.connector.ConnectorTransactionHandle; +import io.prestosql.spi.type.TypeManager; +import org.apache.hadoop.fs.Path; + +import javax.inject.Inject; + +import java.net.URI; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +import static io.prestosql.delta.DeltaExpressionUtils.sanitizePartitionValues; +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.CompletableFuture.completedFuture; + +public class DeltaSplitManager + implements ConnectorSplitManager +{ + private final String connectorId; + private final DeltaConfig deltaConfig; + private final DeltaClient deltaClient; + private final TypeManager typeManager; + + @Inject + public DeltaSplitManager( + DeltaConnectorId connectorId, + DeltaConfig deltaConfig, + DeltaClient deltaClient, + TypeManager typeManager) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); + this.deltaConfig = requireNonNull(deltaConfig, "deltaConfig is null"); + this.deltaClient = requireNonNull(deltaClient, "deltaClient is null"); + this.typeManager = requireNonNull(typeManager, "typeManager is null"); + } + + @Override + public ConnectorSplitSource getSplits( + ConnectorTransactionHandle transaction, + ConnectorSession session, + ConnectorTableHandle table, + SplitSchedulingStrategy splitSchedulingStrategy) + { + DeltaTableHandle tableHandle = (DeltaTableHandle) table; + return new DeltaSplitSource(session, tableHandle); + } + + private class DeltaSplitSource + implements ConnectorSplitSource + { + private final DeltaTable deltaTable; + private final Iterator fileIter; + private final int maxBatchSize; + + DeltaSplitSource(ConnectorSession session, DeltaTableHandle deltaTableHandle) + { + this.deltaTable = deltaTableHandle.getDeltaTable(); + this.fileIter = DeltaExpressionUtils.iterateWithPredicate( + session, + deltaClient.listFiles(session, deltaTable), + deltaTableHandle.getPredicate(), + typeManager); + this.maxBatchSize = deltaConfig.getMaxSplitsBatchSize(); + } + + @Override + public CompletableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize) + { + List splits = new ArrayList<>(); + while (fileIter.hasNext() && splits.size() < maxSize && splits.size() < maxBatchSize) { + AddFile file = fileIter.next(); + Path filePath = new Path(deltaTable.getTableLocation(), URI.create(file.getPath()).getPath()); + splits.add(new DeltaSplit( + connectorId, + deltaTable.getSchemaName(), + deltaTable.getTableName(), + filePath.toString(), + 0, /* start */ + file.getSize() /* split length - read the entire file in one split */, + file.getSize(), + sanitizePartitionValues(file.getPartitionValues()))); + } + + return completedFuture(new ConnectorSplitBatch(splits, !fileIter.hasNext())); + } + + @Override + public void close() + { + } + + @Override + public boolean isFinished() + { + return !fileIter.hasNext(); + } + } +} diff --git a/presto-delta/src/main/java/io/prestosql/delta/DeltaTable.java b/presto-delta/src/main/java/io/prestosql/delta/DeltaTable.java new file mode 100644 index 000000000000..f5ae110ccb75 --- /dev/null +++ b/presto-delta/src/main/java/io/prestosql/delta/DeltaTable.java @@ -0,0 +1,118 @@ +/* + * 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.prestosql.delta; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Strings.isNullOrEmpty; +import static java.util.Objects.requireNonNull; + +public class DeltaTable +{ + private final String schemaName; + private final String tableName; + private final String tableLocation; + private final Optional snapshotId; + private final List columns; + + @JsonCreator + public DeltaTable( + @JsonProperty("schemaName") String schemaName, + @JsonProperty("tableName") String tableName, + @JsonProperty("tableLocation") String tableLocation, + @JsonProperty("snapshotId") Optional snapshotId, + @JsonProperty("columns") List columns) + { + checkArgument(!isNullOrEmpty(schemaName), "schemaName is null or is empty"); + checkArgument(!isNullOrEmpty(tableName), "tableName is null or is empty"); + this.schemaName = schemaName; + this.tableName = tableName; + this.tableLocation = requireNonNull(tableLocation, "tableLocation is null"); + this.snapshotId = requireNonNull(snapshotId, "snapshotId is null"); + this.columns = ImmutableList.copyOf(requireNonNull(columns, "columns is null")); + } + + @JsonProperty + public String getSchemaName() + { + return schemaName; + } + + @JsonProperty + public String getTableName() + { + return tableName; + } + + @JsonProperty + public String getTableLocation() + { + return tableLocation; + } + + @JsonProperty + public Optional getSnapshotId() + { + return snapshotId; + } + + @JsonProperty + public List getColumns() + { + return columns; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + DeltaTable that = (DeltaTable) o; + return Objects.equals(schemaName, that.schemaName) && + Objects.equals(tableName, that.tableName) && + Objects.equals(tableLocation, that.tableLocation) && + Objects.equals(snapshotId, that.snapshotId) && + Objects.equals(columns, that.columns); + } + + @Override + public int hashCode() + { + return Objects.hash(schemaName, tableName, tableLocation, snapshotId, columns); + } + + @Override + public String toString() + { + final StringBuilder sb = new StringBuilder("DeltaTableHandle{"); + sb.append("table='").append(schemaName + "." + tableName).append('\''); + sb.append(", location=").append(tableLocation); + sb.append(", snapshotId=").append(snapshotId); + sb.append(", columns=").append(columns); + sb.append('}'); + return sb.toString(); + } +} diff --git a/presto-delta/src/main/java/io/prestosql/delta/DeltaTableHandle.java b/presto-delta/src/main/java/io/prestosql/delta/DeltaTableHandle.java new file mode 100644 index 000000000000..683571d212de --- /dev/null +++ b/presto-delta/src/main/java/io/prestosql/delta/DeltaTableHandle.java @@ -0,0 +1,120 @@ +/* + * 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.prestosql.delta; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.prestosql.spi.connector.ConnectorTableHandle; +import io.prestosql.spi.connector.SchemaTableName; +import io.prestosql.spi.predicate.TupleDomain; + +import java.util.Objects; +import java.util.Optional; + +import static com.google.common.base.Preconditions.checkState; +import static java.util.Objects.requireNonNull; + +public final class DeltaTableHandle + implements ConnectorTableHandle +{ + private final String connectorId; + private final DeltaTable deltaTable; + private final TupleDomain predicate; + private final Optional predicateString; + + @JsonCreator + public DeltaTableHandle( + @JsonProperty("connectorId") String connectorId, + @JsonProperty("deltaTable") DeltaTable deltaTable, + @JsonProperty("predicate") TupleDomain predicate, + @JsonProperty("predicateString") Optional predicateString) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null"); + this.deltaTable = requireNonNull(deltaTable, "deltaTable is null"); + this.predicate = requireNonNull(predicate, "predicate is null"); + this.predicateString = requireNonNull(predicateString, "predicateString is null"); + } + + DeltaTableHandle withPredicate(TupleDomain predicate, String predicateString) + { + checkState(this.predicate.isAll(), "There is already a predicate."); + return new DeltaTableHandle( + connectorId, + deltaTable, + predicate, + Optional.of(predicateString)); + } + + @JsonProperty + public String getConnectorId() + { + return connectorId; + } + + @JsonProperty + public DeltaTable getDeltaTable() + { + return deltaTable; + } + + @JsonProperty + public TupleDomain getPredicate() + { + return predicate; + } + + @JsonProperty + public Optional getPredicateString() + { + return predicateString; + } + + public SchemaTableName toSchemaTableName() + { + return new SchemaTableName(deltaTable.getSchemaName(), deltaTable.getTableName()); + } + + @Override + public int hashCode() + { + return Objects.hash(connectorId, deltaTable, predicate); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if ((obj == null) || (getClass() != obj.getClass())) { + return false; + } + + DeltaTableHandle other = (DeltaTableHandle) obj; + return Objects.equals(this.connectorId, other.connectorId) && + Objects.equals(this.deltaTable, other.deltaTable) && + Objects.equals(this.predicate, other.predicate); + } + + @Override + public String toString() + { + final StringBuilder sb = new StringBuilder("DeltaTableHandle{"); + sb.append("connectorId='").append(connectorId).append('\''); + sb.append(", table='").append(toSchemaTableName()).append('\''); + sb.append(", predicate=").append(predicateString); + sb.append('}'); + return sb.toString(); + } +} diff --git a/presto-delta/src/main/java/io/prestosql/delta/DeltaTableName.java b/presto-delta/src/main/java/io/prestosql/delta/DeltaTableName.java new file mode 100644 index 000000000000..de504ac45709 --- /dev/null +++ b/presto-delta/src/main/java/io/prestosql/delta/DeltaTableName.java @@ -0,0 +1,143 @@ +/* + * 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.prestosql.delta; + +import io.prestosql.spi.PrestoException; + +import java.sql.Timestamp; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; +import java.time.format.DateTimeParseException; +import java.time.temporal.ChronoField; +import java.util.Optional; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import static io.prestosql.spi.StandardErrorCode.NOT_SUPPORTED; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +/** + * Delta table name. Supported table name formats: + *
    + *
  • table name. E.g. SELECT * FROM delta.db.sales
  • + *
  • path to the Delta table location. E.g. SELECT * FROM "s3://bucket/path/delta-table"
  • + *
+ *

+ * Table name can have a suffix to indicate reading a particular snapshot of the table + *

    + *
  • Snapshot version. E.g. SELECT * FROM delta.db."sales@v123" -- to read 123 snapshot version
  • + *
  • Latest snapshot as of given timestamp. E.g. SELECT * FROM delta.db."sales@t2021-11-18 10:30:00" + * -- to read latest snapshot on or before 2021-11-18 10:30:00
  • + *
+ */ +public class DeltaTableName +{ + private static final Pattern TABLE_PATTERN = Pattern.compile( + "(?
[^@]+)" + /* matches table name that doesn't contain `@` character */ + "(?:@v(?[0-9]+))?" + + "(?:@t(?[0-9.\\-: ]+))?"); + + private static final DateTimeFormatter TIMESTAMP_PARSER = + new DateTimeFormatterBuilder().appendPattern("yyyy[-MM[-dd[ HH[:mm[:ss]]]]]") + .parseDefaulting(ChronoField.MONTH_OF_YEAR, 1) + .parseDefaulting(ChronoField.DAY_OF_MONTH, 1) + .parseDefaulting(ChronoField.HOUR_OF_DAY, 0) + .parseDefaulting(ChronoField.MINUTE_OF_HOUR, 0) + .parseDefaulting(ChronoField.SECOND_OF_MINUTE, 0) + .toFormatter() + .withZone(ZoneId.of("UTC")); + + private final String tableNameOrPath; + private final Optional snapshotId; + private final Optional snapshotAsOfTimestamp; + + public DeltaTableName(String tableNameOrPath, Optional snapshotId, Optional snapshotAsOfTimestamp) + { + this.tableNameOrPath = requireNonNull(tableNameOrPath, "tableNameOrPath is null"); + this.snapshotId = requireNonNull(snapshotId, "snapshotId is null"); + this.snapshotAsOfTimestamp = requireNonNull(snapshotAsOfTimestamp, "snapshotAsOfTimestamp is null"); + } + + public String getTableNameOrPath() + { + return tableNameOrPath; + } + + public Optional getSnapshotId() + { + return snapshotId; + } + + public Optional getSnapshotAsOfTimestamp() + { + return snapshotAsOfTimestamp; + } + + @Override + public String toString() + { + StringBuilder builder = new StringBuilder() + .append("Table[" + tableNameOrPath + "]"); + snapshotId.map(id -> builder.append("@v" + id)); + snapshotAsOfTimestamp.map(ts -> builder.append("@t" + new Timestamp(ts))); + return builder.toString(); + } + + public static DeltaTableName from(String tableName) + { + Matcher match = TABLE_PATTERN.matcher(tableName); + if (!match.matches()) { + throw new PrestoException(NOT_SUPPORTED, "Invalid Delta table name: " + tableName + + ", Expected table name form 'tableName[@v][@t]'. " + + "The table can have either a particular snapshot identifier or a timestamp of the snapshot. " + + "If timestamp is given the latest snapshot of the table that was generated at or " + + "before the given timestamp is read"); + } + + String tableNameOrPath = match.group("table"); + String snapshotValue = match.group("snapshotId"); + Optional snapshot = Optional.empty(); + if (snapshotValue != null) { + snapshot = Optional.of(Long.parseLong(snapshotValue)); + } + + Optional timestampMillisUtc = Optional.empty(); + String timestampValue = match.group("timestamp"); + if (timestampValue != null) { + try { + timestampMillisUtc = Optional.of( + LocalDateTime.from(TIMESTAMP_PARSER.parse(timestampValue)) + .toInstant(ZoneOffset.UTC) + .toEpochMilli()); + } + catch (IllegalArgumentException | DateTimeParseException ex) { + throw new PrestoException(NOT_SUPPORTED, + format("Invalid Delta table name: %s, given snapshot timestamp (%s) format is not valid. " + + "Expected timestamp format 'YYYY-MM-DD HH:mm:ss'", tableName, timestampValue)); + } + } + + if (snapshot.isPresent() && timestampMillisUtc.isPresent()) { + throw new PrestoException(NOT_SUPPORTED, "Invalid Delta table name: " + tableName + + ", Table suffix contains both snapshot id and timestamp of snapshot to read. " + + "Only one of them is supported."); + } + + return new DeltaTableName(tableNameOrPath, snapshot, timestampMillisUtc); + } +} diff --git a/presto-delta/src/main/java/io/prestosql/delta/DeltaTransactionHandle.java b/presto-delta/src/main/java/io/prestosql/delta/DeltaTransactionHandle.java new file mode 100644 index 000000000000..a90a940c1032 --- /dev/null +++ b/presto-delta/src/main/java/io/prestosql/delta/DeltaTransactionHandle.java @@ -0,0 +1,22 @@ +/* + * 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.prestosql.delta; + +import io.prestosql.spi.connector.ConnectorTransactionHandle; + +public enum DeltaTransactionHandle + implements ConnectorTransactionHandle +{ + INSTANCE +} diff --git a/presto-delta/src/main/java/io/prestosql/delta/DeltaTypeUtils.java b/presto-delta/src/main/java/io/prestosql/delta/DeltaTypeUtils.java new file mode 100644 index 000000000000..85b3bac346fc --- /dev/null +++ b/presto-delta/src/main/java/io/prestosql/delta/DeltaTypeUtils.java @@ -0,0 +1,229 @@ +/* + * 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.prestosql.delta; + +import com.google.common.collect.ImmutableList; +import io.delta.standalone.types.ArrayType; +import io.delta.standalone.types.BinaryType; +import io.delta.standalone.types.BooleanType; +import io.delta.standalone.types.ByteType; +import io.delta.standalone.types.DataType; +import io.delta.standalone.types.DateType; +import io.delta.standalone.types.DecimalType; +import io.delta.standalone.types.DoubleType; +import io.delta.standalone.types.FloatType; +import io.delta.standalone.types.IntegerType; +import io.delta.standalone.types.LongType; +import io.delta.standalone.types.MapType; +import io.delta.standalone.types.ShortType; +import io.delta.standalone.types.StringType; +import io.delta.standalone.types.StructType; +import io.delta.standalone.types.TimestampType; +import io.prestosql.spi.PrestoException; +import io.prestosql.spi.connector.SchemaTableName; +import io.prestosql.spi.type.Decimals; +import io.prestosql.spi.type.NamedTypeSignature; +import io.prestosql.spi.type.RowFieldName; +import io.prestosql.spi.type.StandardTypes; +import io.prestosql.spi.type.Type; +import io.prestosql.spi.type.TypeSignature; +import io.prestosql.spi.type.TypeSignatureParameter; + +import java.sql.Timestamp; +import java.time.LocalDate; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeParseException; +import java.util.Arrays; +import java.util.Locale; +import java.util.Optional; + +import static com.google.common.base.Preconditions.checkArgument; +import static io.airlift.slice.Slices.utf8Slice; +import static io.prestosql.delta.DeltaErrorCode.DELTA_INVALID_PARTITION_VALUE; +import static io.prestosql.delta.DeltaErrorCode.DELTA_UNSUPPORTED_COLUMN_TYPE; +import static io.prestosql.spi.type.BigintType.BIGINT; +import static io.prestosql.spi.type.BooleanType.BOOLEAN; +import static io.prestosql.spi.type.DateType.DATE; +import static io.prestosql.spi.type.DecimalType.createDecimalType; +import static io.prestosql.spi.type.DoubleType.DOUBLE; +import static io.prestosql.spi.type.IntegerType.INTEGER; +import static io.prestosql.spi.type.RealType.REAL; +import static io.prestosql.spi.type.SmallintType.SMALLINT; +import static io.prestosql.spi.type.StandardTypes.ARRAY; +import static io.prestosql.spi.type.StandardTypes.MAP; +import static io.prestosql.spi.type.StandardTypes.ROW; +import static io.prestosql.spi.type.TimestampType.TIMESTAMP; +import static io.prestosql.spi.type.TinyintType.TINYINT; +import static io.prestosql.spi.type.VarbinaryType.VARBINARY; +import static io.prestosql.spi.type.VarcharType.createUnboundedVarcharType; +import static java.lang.Double.parseDouble; +import static java.lang.Float.floatToRawIntBits; +import static java.lang.Float.parseFloat; +import static java.lang.Long.parseLong; +import static java.lang.String.format; + +/** + * Contains utility methods to convert Delta data types (and data values) to Presto data types (and data values) + */ +public class DeltaTypeUtils +{ + private DeltaTypeUtils() + { + } + + /** + * Convert given Delta data type to Presto data type signature. + * + * @param tableName Used in error messages when an unsupported data type is encountered. + * @param columnName Used in error messages when an unsupported data type is encountered. + * @param deltaType Data type to convert + * @return + */ + public static TypeSignature convertDeltaDataTypePrestoDataType(SchemaTableName tableName, String columnName, DataType deltaType) + { + checkArgument(deltaType != null); + + if (deltaType instanceof StructType) { + StructType deltaStructType = (StructType) deltaType; + ImmutableList.Builder typeSignatureBuilder = ImmutableList.builder(); + Arrays.stream(deltaStructType.getFields()) + .forEach(field -> { + String rowFieldName = field.getName().toLowerCase(Locale.US); + TypeSignature rowFieldType = convertDeltaDataTypePrestoDataType( + tableName, + columnName + "." + field.getName(), + field.getDataType()); + typeSignatureBuilder.add( + TypeSignatureParameter.namedTypeParameter( + new NamedTypeSignature( + Optional.of(new RowFieldName(rowFieldName)), + rowFieldType))); + }); + return new TypeSignature(ROW, typeSignatureBuilder.build()); + } + else if (deltaType instanceof ArrayType) { + ArrayType deltaArrayType = (ArrayType) deltaType; + TypeSignature elementType = convertDeltaDataTypePrestoDataType( + tableName, + columnName, + deltaArrayType.getElementType()); + return new TypeSignature( + ARRAY, + ImmutableList.of(TypeSignatureParameter.typeParameter(elementType))); + } + else if (deltaType instanceof MapType) { + MapType deltaMapType = (MapType) deltaType; + TypeSignature keyType = convertDeltaDataTypePrestoDataType(tableName, columnName, deltaMapType.getKeyType()); + TypeSignature valueType = convertDeltaDataTypePrestoDataType(tableName, columnName, deltaMapType.getValueType()); + return new TypeSignature(MAP, ImmutableList.of( + TypeSignatureParameter.typeParameter(keyType), + TypeSignatureParameter.typeParameter(valueType))); + } + + return convertDeltaPrimitiveTypeToPrestoPrimitiveType(tableName, columnName, deltaType).getTypeSignature(); + } + + public static Object convertPartitionValue( + String partitionColumnName, + String partitionValue, + TypeSignature partitionDataType) + { + if (partitionValue == null) { + return null; + } + + String typeBase = partitionDataType.getBase(); + try { + switch (typeBase) { + case StandardTypes.TINYINT: + case StandardTypes.SMALLINT: + case StandardTypes.INTEGER: + case StandardTypes.BIGINT: + return parseLong(partitionValue); + case StandardTypes.REAL: + return (long) floatToRawIntBits(parseFloat(partitionValue)); + case StandardTypes.DOUBLE: + return parseDouble(partitionValue); + case StandardTypes.VARCHAR: + case StandardTypes.VARBINARY: + return utf8Slice(partitionValue); + case StandardTypes.DATE: + return LocalDate.parse(partitionValue, DateTimeFormatter.ISO_LOCAL_DATE).toEpochDay(); + case StandardTypes.TIMESTAMP: + return Timestamp.valueOf(partitionValue).toLocalDateTime().toEpochSecond(ZoneOffset.UTC) * 1000; + case StandardTypes.BOOLEAN: + checkArgument(partitionValue.equalsIgnoreCase("true") || partitionValue.equalsIgnoreCase("false")); + return Boolean.valueOf(partitionValue); + case StandardTypes.DECIMAL: + return Decimals.parse(partitionValue).getObject(); + default: + throw new PrestoException(DELTA_UNSUPPORTED_COLUMN_TYPE, + format("Unsupported data type '%s' for partition column %s", partitionDataType, partitionColumnName)); + } + } + catch (IllegalArgumentException | DateTimeParseException e) { + throw new PrestoException(DELTA_INVALID_PARTITION_VALUE, + format("Can not parse partition value '%s' of type '%s' for partition column '%s'", + partitionValue, partitionDataType, partitionColumnName)); + } + } + + private static Type convertDeltaPrimitiveTypeToPrestoPrimitiveType(SchemaTableName tableName, String columnName, DataType deltaType) + { + if (deltaType instanceof BinaryType) { + return VARBINARY; + } + else if (deltaType instanceof BooleanType) { + return BOOLEAN; + } + else if (deltaType instanceof ByteType) { + return TINYINT; + } + else if (deltaType instanceof DateType) { + return DATE; + } + else if (deltaType instanceof DecimalType) { + DecimalType decimalType = (DecimalType) deltaType; + return createDecimalType(decimalType.getPrecision(), decimalType.getScale()); + } + else if (deltaType instanceof DoubleType) { + return DOUBLE; + } + else if (deltaType instanceof FloatType) { + return REAL; + } + else if (deltaType instanceof IntegerType) { + return INTEGER; + } + else if (deltaType instanceof LongType) { + return BIGINT; + } + else if (deltaType instanceof ShortType) { + return SMALLINT; + } + else if (deltaType instanceof StringType) { + return createUnboundedVarcharType(); + } + else if (deltaType instanceof TimestampType) { + return TIMESTAMP; + } + + throw new PrestoException(DELTA_UNSUPPORTED_COLUMN_TYPE, + format("Column '%s' in Delta table %s contains unsupported data type: %s", + columnName, + tableName, + deltaType.getCatalogString())); + } +} diff --git a/presto-delta/src/test/java/io/prestosql/delta/AbstractDeltaDistributedQueryTestBase.java b/presto-delta/src/test/java/io/prestosql/delta/AbstractDeltaDistributedQueryTestBase.java new file mode 100644 index 000000000000..4888084970fe --- /dev/null +++ b/presto-delta/src/test/java/io/prestosql/delta/AbstractDeltaDistributedQueryTestBase.java @@ -0,0 +1,155 @@ +/* + * 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.prestosql.delta; + +import com.google.common.collect.ImmutableMap; +import io.prestosql.Session; +import io.prestosql.plugin.hive.HiveHadoop2Plugin; +import io.prestosql.plugin.tpch.TpchPlugin; +import io.prestosql.spi.type.TimeZoneKey; +import io.prestosql.testing.AbstractTestQueryFramework; +import io.prestosql.testing.DistributedQueryRunner; +import io.prestosql.testing.QueryRunner; + +import java.nio.file.Path; +import java.util.Locale; +import java.util.Map; + +import static io.prestosql.testing.TestingSession.testSessionBuilder; +import static java.lang.String.format; + +public abstract class AbstractDeltaDistributedQueryTestBase + extends AbstractTestQueryFramework +{ + public static final String DELTA_CATALOG = "delta"; + public static final String HIVE_CATALOG = "hive"; + public static final String PATH_SCHEMA = "$path$"; + public static final String DELTA_SCHEMA = "deltaTables"; // Schema in Hive which has test Delta tables + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + QueryRunner queryRunner = createDeltaQueryRunner(ImmutableMap.of()); + return queryRunner; + } + + /** + * Helper method to + * - registering Delta table given location as a Hive table and + * - run the test query, expected results query and compare the results. + */ + protected void assertDeltaQuery(String deltaTable, String testQuery, String expResultsQuery) + { + try { + registerDeltaTableInHMS(deltaTable, deltaTable); + assertQuery(testQuery, expResultsQuery); + } + finally { + unregisterDeltaTableInHMS(deltaTable); + } + } + + /** + * Helper method to + * - registering Delta table given location as a Hive table and + * - run the test query, expect the query to fail and failure contains the expected message + */ + protected void assertDeltaQueryFails(String deltaTable, String testQuery, String expFailure) + { + try { + registerDeltaTableInHMS(deltaTable, deltaTable); + assertQueryFails(testQuery, expFailure); + } + finally { + unregisterDeltaTableInHMS(deltaTable); + } + } + + protected static String goldenTablePath(String tableName) + { + return AbstractDeltaDistributedQueryTestBase.class.getClassLoader().getResource(tableName).toString(); + } + + private static DistributedQueryRunner createDeltaQueryRunner(Map extraProperties) + throws Exception + { + Session session = testSessionBuilder() + .setCatalog(DELTA_CATALOG) + .setSchema(DELTA_SCHEMA.toLowerCase(Locale.ROOT)) + .setTimeZoneKey(TimeZoneKey.UTC_KEY) + .build(); + + DistributedQueryRunner queryRunner = DistributedQueryRunner.builder(session) + .setExtraProperties(extraProperties) + .build(); + + // Install the TPCH plugin for test data (not in Delta format) + queryRunner.installPlugin(new TpchPlugin()); + queryRunner.createCatalog("tpch", "tpch"); + + Path dataDir = queryRunner.getCoordinator().getBaseDataDir().resolve("delta_metadata"); + Path catalogDir = dataDir.getParent().resolve("catalog"); + + // Install a Delta connector catalog + queryRunner.installPlugin(new DeltaPlugin()); + Map deltaProperties = ImmutableMap.builder() + .put("hive.metastore", "file") + .put("hive.metastore.catalog.dir", catalogDir.toFile().toURI().toString()) + .build(); + queryRunner.createCatalog(DELTA_CATALOG, "delta", deltaProperties); + + // Install a Hive connector catalog that uses the same metastore as Delta + // This catalog will be used to create tables in metastore as the Delta connector doesn't + // support creating tables yet. + queryRunner.installPlugin(new HiveHadoop2Plugin()); + Map hiveProperties = ImmutableMap.builder() + .put("hive.metastore", "file") + .put("hive.metastore.catalog.dir", catalogDir.toFile().toURI().toString()) + .put("hive.allow-drop-table", "true") + .put("hive.security", "legacy") + .build(); + queryRunner.createCatalog(HIVE_CATALOG, "hive-hadoop2", hiveProperties); + queryRunner.execute(format("CREATE SCHEMA %s.%s", HIVE_CATALOG, DELTA_SCHEMA)); + + return queryRunner; + } + + /** + * Register the given deltaTableName as hiveTableName in HMS using the Hive storage catalog. + * Hive and Delta catalogs share the same HMS in this test. Hive is used to register the tables as Delta + * connector doesn't have the write support yet. + * + * @param deltaTableName Name of the delta table which is on the classpath. + * @param hiveTableName Name of the Hive table that the Delta table is to be registered as in HMS + */ + protected void registerDeltaTableInHMS(String deltaTableName, String hiveTableName) + { + getQueryRunner().execute(format( + "CREATE TABLE %s.\"%s\".\"%s\" (dummyColumn INT) WITH (external_location = '%s')", + HIVE_CATALOG, + DELTA_SCHEMA, + hiveTableName, + goldenTablePath(deltaTableName))); + } + + /** + * Drop the given table from HMS + */ + protected void unregisterDeltaTableInHMS(String hiveTableName) + { + getQueryRunner().execute( + format("DROP TABLE IF EXISTS %s.\"%s\".\"%s\"", HIVE_CATALOG, DELTA_SCHEMA, hiveTableName)); + } +} diff --git a/presto-delta/src/test/java/io/prestosql/delta/TestDeltaColumnHandle.java b/presto-delta/src/test/java/io/prestosql/delta/TestDeltaColumnHandle.java new file mode 100644 index 000000000000..bf4a0ad8e49c --- /dev/null +++ b/presto-delta/src/test/java/io/prestosql/delta/TestDeltaColumnHandle.java @@ -0,0 +1,74 @@ +/* + * 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.prestosql.delta; + +import com.google.common.collect.ImmutableMap; +import io.airlift.json.JsonCodec; +import io.airlift.json.JsonCodecFactory; +import io.airlift.json.ObjectMapperProvider; +import io.prestosql.spi.type.TypeSignature; +import io.prestosql.type.TypeSignatureDeserializer; +import org.testng.annotations.Test; + +import static io.prestosql.delta.DeltaColumnHandle.ColumnType.PARTITION; +import static io.prestosql.delta.DeltaColumnHandle.ColumnType.REGULAR; +import static io.prestosql.spi.type.DoubleType.DOUBLE; +import static org.testng.Assert.assertEquals; + +/** + * Test {@link DeltaColumnHandle} is created with correct parameters and round trip JSON SerDe works. + */ +public class TestDeltaColumnHandle +{ + private final JsonCodec codec = + JsonCodec.jsonCodec(DeltaColumnHandle.class); + + @Test + public void testRegularColumn() + { + DeltaColumnHandle expectedPartitionColumn = new DeltaColumnHandle( + "name", + "partCol", + DOUBLE.getTypeSignature(), + PARTITION); + testRoundTrip(expectedPartitionColumn); + } + + @Test + public void testPartitionKeyColumn() + { + DeltaColumnHandle expectedRegularColumn = new DeltaColumnHandle( + "name", + "regularCol", + DOUBLE.getTypeSignature(), + REGULAR); + testRoundTrip(expectedRegularColumn); + } + + private void testRoundTrip(DeltaColumnHandle expected) + { + ObjectMapperProvider provider = new ObjectMapperProvider(); + provider.setJsonDeserializers(ImmutableMap.of( + TypeSignature.class, new TypeSignatureDeserializer())); + JsonCodec codec = new JsonCodecFactory(provider) + .jsonCodec(DeltaColumnHandle.class); + String json = codec.toJson(expected); + DeltaColumnHandle actual = codec.fromJson(json); + + assertEquals(actual.getConnectorId(), expected.getConnectorId()); + assertEquals(actual.getName(), expected.getName()); + assertEquals(actual.getColumnType(), expected.getColumnType()); + assertEquals(actual.getDataType(), expected.getDataType()); + } +} diff --git a/presto-delta/src/test/java/io/prestosql/delta/TestDeltaConfig.java b/presto-delta/src/test/java/io/prestosql/delta/TestDeltaConfig.java new file mode 100644 index 000000000000..b1903922c7f0 --- /dev/null +++ b/presto-delta/src/test/java/io/prestosql/delta/TestDeltaConfig.java @@ -0,0 +1,52 @@ +/* + * 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.prestosql.delta; + +import com.google.common.collect.ImmutableMap; +import io.airlift.configuration.testing.ConfigAssertions; +import org.testng.annotations.Test; + +import java.util.Map; + +public class TestDeltaConfig +{ + @Test + public void testDefaults() + { + ConfigAssertions.assertRecordedDefaults(ConfigAssertions.recordDefaults(DeltaConfig.class) + .setPartitionPruningEnabled(true) + .setFilterPushdownEnabled(true) + .setMaxSplitsBatchSize(200) + .setProjectionPushdownEnabled(true)); + } + + @Test + public void testExplicitPropertyMappings() + { + Map properties = new ImmutableMap.Builder() + .put("delta.partition-pruning-enabled", "false") + .put("delta.filter-pushdown-enabled", "false") + .put("delta.max-splits-batch-size", "400") + .put("delta.projection-pushdown-enabled", "false") + .build(); + + DeltaConfig expected = new DeltaConfig() + .setFilterPushdownEnabled(false) + .setPartitionPruningEnabled(false) + .setMaxSplitsBatchSize(400) + .setProjectionPushdownEnabled(false); + + ConfigAssertions.assertFullMapping(properties, expected); + } +} diff --git a/presto-delta/src/test/java/io/prestosql/delta/TestDeltaIntegration.java b/presto-delta/src/test/java/io/prestosql/delta/TestDeltaIntegration.java new file mode 100644 index 000000000000..67ae1cbe1840 --- /dev/null +++ b/presto-delta/src/test/java/io/prestosql/delta/TestDeltaIntegration.java @@ -0,0 +1,239 @@ +/* + * 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.prestosql.delta; + +import com.google.common.base.Joiner; +import org.testng.annotations.Test; + +import java.util.ArrayList; +import java.util.List; + +import static java.lang.String.format; + +/** + * Integration tests for reading Delta tables. + */ +public class TestDeltaIntegration + extends AbstractDeltaDistributedQueryTestBase +{ + @Test + public void readPrimitiveTypeData() + { + // Test reading following primitive types from a Delta table (all ints, float, double, decimal, boolean, varchar, varbinary) + String testQuery = + format("SELECT * FROM \"%s\".\"%s\"", PATH_SCHEMA, goldenTablePath("data-reader-primitives")); + String expResultsQuery = getPrimitiveTypeTableData(); + assertQuery(testQuery, expResultsQuery); + } + + @Test + public void readArrayTypeData() + { + // Test reading following array elements with type (all ints, float, double, decimal, boolean, varchar, varbinary) + String testQuery = + format("SELECT * FROM \"%s\".\"%s\"", PATH_SCHEMA, goldenTablePath("data-reader-array-primitives")); + + // Create query for the expected results. + List expRows = new ArrayList<>(); + for (byte i = 0; i < 10; i++) { + expRows.add(format("SELECT " + + " array[cast(%s as integer)]," + + " array[cast(%s as bigint)]," + + " array[cast(%s as tinyint)]," + + " array[cast(%s as smallint)]," + + " array[%s]," + + " array[cast(%s as real)]," + + " array[cast(%s as double)], " + + " array['%s'], " + + " array[cast(X'0%s0%s' as varbinary)], " + + " array[cast(%s as decimal)]", i, i, i, i, (i % 2 == 0 ? "true" : "false"), i, i, i, i, i, i)); + } + String expResultsQuery = Joiner.on(" UNION ").join(expRows); + + assertQuery(testQuery, expResultsQuery); + } + + @Test + public void readMapTypeData() + { + // Test reading MAP data type columns from Delta table + String testQuery = + format("SELECT map_keys(a), map_values(e) FROM \"%s\".\"%s\"", PATH_SCHEMA, goldenTablePath("data-reader-map")); + + List expRows = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + expRows.add("SELECT " + + " ARRAY[cast(" + i + " as integer)]," + + " ARRAY[cast(" + i + " as decimal)]"); + } + String expResultsQuery = Joiner.on(" UNION ").join(expRows); + + assertQuery(testQuery, expResultsQuery); + } + + @Test + void readTableRegisteredInHMS() + { + String deltaTable = "data-reader-primitives"; + String testQuery = format("SELECT * FROM \"%s\"", deltaTable); + String expResultsQuery = getPrimitiveTypeTableData(); + assertDeltaQuery(deltaTable, testQuery, expResultsQuery); + } + + @Test + void readSpecificSnapshotVersion() + { + String deltaTable = "snapshot-data3"; + String testQueryTemplate = "SELECT * FROM \"%s@%s\" WHERE col1 = 0"; + + // read snapshot version 2 + String testQueryV2 = format(testQueryTemplate, deltaTable, "v2"); + String expResultsQueryV2 = "SELECT * FROM VALUES(0, 'data-2-0')"; + assertDeltaQuery(deltaTable, testQueryV2, expResultsQueryV2); + + // read snapshot version 3 + String testQueryV3 = format(testQueryTemplate, deltaTable, "v3"); + String expResultsQueryV3 = "SELECT * FROM VALUES(0, 'data-2-0'), (0, 'data-3-0')"; + assertDeltaQuery(deltaTable, testQueryV3, expResultsQueryV3); + + // All snapshots are created at 2020-10-26 02:52:47.515 + // read snapshot as of 2020-10-26 02:50:00 - this should fail as there are no snapshots before this timestamp + String testQueryTs1 = format(testQueryTemplate, deltaTable, "t2020-10-27 02:50:00"); + assertDeltaQueryFails( + deltaTable, + testQueryTs1, + "There is no snapshot exists in Delta table 'deltatables.snapshot-data3@t2020-10-27 02:50:00' " + + "that is created on or before '2020-10-27T02:50:00Z'"); + + // All snapshots are created at 2020-10-26 02:52:47.515 + // read snapshot as of 2020-10-26 02:52:48 - this should read the latest data + String testQueryTs2 = format(testQueryTemplate, deltaTable, "t2020-10-26 02:52:48"); + String expResultsQueryTs2 = "SELECT * FROM VALUES(0, 'data-2-0'), (0, 'data-3-0')"; + assertDeltaQuery(deltaTable, testQueryTs2, expResultsQueryTs2); + } + + @Test + void readCheckpointedDeltaTable() + { + // Delta table commits are periodically checkpointed into a parquet file. + // Test Delta connector is able to read the checkpointed commits in a parquet file. + // Test table has commit files (0-10) deleted. So it has to rely on reading the Parquet file + // to fetch the files latest commit (i.e > 10). + String deltaTable = "checkpointed-delta-table"; + String testQueryTemplate = "SELECT * FROM \"%s%s\" WHERE col1 in (0, 10, 15)"; + + // read snapshot version 3 - expect can't time travel error + String testQueryV3 = format(testQueryTemplate, deltaTable, "@v3"); + assertDeltaQueryFails( + deltaTable, + testQueryV3, + "Can not find snapshot \\(3\\) in Delta table 'deltatables.checkpointed-delta-table\\@v3': No reproducible commits found at .*"); + + // read latest data + String testQueryLatest = format(testQueryTemplate, deltaTable, ""); + String expResultsQueryLatest = "SELECT * FROM VALUES(0, 'data-0-0'), (10, 'data-0-10'), (15, 'data-0-15')"; + assertDeltaQuery(deltaTable, testQueryLatest, expResultsQueryLatest); + + // read snapshot version 13 + String testQueryV13 = format(testQueryTemplate, deltaTable, "@v13"); + String expResultsQueryV13 = "SELECT * FROM VALUES(0, 'data-0-0'), (10, 'data-0-10')"; + assertDeltaQuery(deltaTable, testQueryV13, expResultsQueryV13); + } + + @Test + public void readPartitionedTable() + { + String deltaTable = "time-travel-partition-changes-b"; + String testQuery1 = format("SELECT * FROM \"%s\" WHERE id in (10, 15, 12, 13)", deltaTable); + String expResultsQuery1 = "SELECT * FROM VALUES(10, 0),(15, 1),(12, 0),(13, 1)"; + assertDeltaQuery(deltaTable, testQuery1, expResultsQuery1); + + // reorder the columns in output and query the partitioned table + String testQuery2 = format("SELECT part2, id FROM \"%s\" WHERE id in (16, 14, 19)", deltaTable); + String expResultsQuery2 = "SELECT * FROM VALUES(0, 16),(0, 14),(1, 19)"; + assertDeltaQuery(deltaTable, testQuery2, expResultsQuery2); + } + + @Test + public void readPartitionedTableAllDataTypes() + { + String deltaTable = "data-reader-partition-values"; + String testQuery = format("SELECT * FROM \"%s\"", deltaTable); + String expResultsQuery = "SELECT * FROM VALUES" + + "( 0," + + " cast(0 as bigint)," + + " cast(0 as smallint), " + + " cast(0 as tinyint), " + + " true, " + + " 0.0, " + + " cast(0.0 as double), " + + " '0', " + + " DATE '2021-09-08', " + + " TIMESTAMP '2021-09-08 11:11:11', " + + " cast(0 as decimal)," + + " '0'" + // regular column + "), " + + "( 1," + + " cast(1 as bigint)," + + " cast(1 as smallint), " + + " cast(1 as tinyint), " + + " false, " + + " 1.0, " + + " cast(1.0 as double), " + + " '1', " + + " DATE '2021-09-08', " + + " TIMESTAMP '2021-09-08 11:11:11', " + + " cast(1 as decimal), " + + " '1'" + // regular column + "), " + + "( null," + + " null," + + " null, " + + " null, " + + " null, " + + " null, " + + " null, " + + " null, " + + " null, " + + " null, " + + " null, " + + " '2'" + // regular column + ")"; + assertDeltaQuery(deltaTable, testQuery, expResultsQuery); + } + + /** + * Expected results for table "data-reader-primitives" + */ + private static String getPrimitiveTypeTableData() + { + // Create query for the expected results. + List expRows = new ArrayList<>(); + for (byte i = 0; i < 10; i++) { + expRows.add(format("SELECT " + + " cast(%s as integer)," + + " cast(%s as bigint)," + + " cast(%s as tinyint)," + + " cast(%s as smallint)," + + " %s," + + " cast(%s as real)," + + " cast(%s as double), " + + " '%s', " + + " cast(X'0%s0%s' as varbinary), " + + " cast(%s as decimal)", i, i, i, i, (i % 2 == 0 ? "true" : "false"), i, i, i, i, i, i)); + } + expRows.add("SELECT null, null, null, null, null, null, null, null, null, null"); + return Joiner.on(" UNION ").join(expRows); + } +} diff --git a/presto-delta/src/test/java/io/prestosql/delta/TestDeltaOptimizationsTests.java b/presto-delta/src/test/java/io/prestosql/delta/TestDeltaOptimizationsTests.java new file mode 100644 index 000000000000..1af4d01b2189 --- /dev/null +++ b/presto-delta/src/test/java/io/prestosql/delta/TestDeltaOptimizationsTests.java @@ -0,0 +1,254 @@ +/* + * 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.prestosql.delta; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.prestosql.Session; +import io.prestosql.cost.PlanNodeStatsEstimate; +import io.prestosql.cost.StatsProvider; +import io.prestosql.execution.warnings.WarningCollector; +import io.prestosql.metadata.Metadata; +import io.prestosql.spi.predicate.Domain; +import io.prestosql.spi.predicate.Range; +import io.prestosql.spi.predicate.SortedRangeSet; +import io.prestosql.sql.planner.Plan; +import io.prestosql.sql.planner.assertions.MatchResult; +import io.prestosql.sql.planner.assertions.Matcher; +import io.prestosql.sql.planner.assertions.PlanMatchPattern; +import io.prestosql.sql.planner.assertions.SymbolAliases; +import io.prestosql.sql.planner.plan.PlanNode; +import io.prestosql.sql.planner.plan.TableScanNode; +import org.testng.annotations.Test; + +import java.util.Collections; +import java.util.Map; + +import static io.airlift.slice.Slices.utf8Slice; +import static io.prestosql.spi.predicate.Domain.multipleValues; +import static io.prestosql.spi.predicate.Domain.notNull; +import static io.prestosql.spi.predicate.Domain.onlyNull; +import static io.prestosql.spi.predicate.Domain.singleValue; +import static io.prestosql.spi.type.IntegerType.INTEGER; +import static io.prestosql.spi.type.VarcharType.VARCHAR; +import static io.prestosql.sql.planner.assertions.MatchResult.NO_MATCH; +import static io.prestosql.sql.planner.assertions.MatchResult.match; +import static io.prestosql.sql.planner.assertions.PlanAssert.assertPlan; +import static io.prestosql.sql.planner.assertions.PlanMatchPattern.anyTree; +import static io.prestosql.transaction.TransactionBuilder.transaction; +import static java.lang.String.format; + +/** + * Integrations tests for various optimization (such as filter pushdown, nested column project/filter pushdown etc) + * that speed up reading data from Delta tables. + */ +public class TestDeltaOptimizationsTests + extends AbstractDeltaDistributedQueryTestBase +{ + @Test + public void filterOnRegularColumn() + { + String tableName = "data-reader-primitives"; + String testQuery = format("SELECT as_int, as_string FROM \"%s\" WHERE as_int = 1", tableName); + String expResultsQuery = "SELECT 1, cast('1' as varchar)"; + + assertDeltaQueryOptimized( + tableName, + testQuery, + expResultsQuery, + ImmutableMap.of("as_int", singleValue(INTEGER, 1L)), + Collections.emptyMap()); + } + + @Test + public void filterOnPartitionColumn() + { + String tableName = "deltatbl-partition-prune"; + String testQuery = format("SELECT date, name, city, cnt FROM \"%s\" WHERE city in ('sh', 'sz')", tableName); + String expResultsQuery = "SELECT * FROM VALUES('20180512', 'Jay', 'sh', 4),('20181212', 'Linda', 'sz', 8)"; + + assertDeltaQueryOptimized( + tableName, + testQuery, + expResultsQuery, + ImmutableMap.of("city", multipleValues(VARCHAR, ImmutableList.of(utf8Slice("sh"), utf8Slice("sz")))), + ImmutableMap.of("city", multipleValues(VARCHAR, ImmutableList.of(utf8Slice("sh"), utf8Slice("sz"))))); + } + + @Test + public void filterOnMultiplePartitionColumns() + { + String tableName = "deltatbl-partition-prune"; + String testQuery = + format("SELECT date, name, city, cnt FROM \"%s\" WHERE city in ('sh', 'sz') AND \"date\" = '20180512'", tableName); + String expResultsQuery = "SELECT * FROM VALUES('20180512', 'Jay', 'sh', 4)"; + + assertDeltaQueryOptimized( + tableName, + testQuery, + expResultsQuery, + ImmutableMap.of( + "city", multipleValues(VARCHAR, ImmutableList.of(utf8Slice("sh"), utf8Slice("sz"))), + "date", singleValue(VARCHAR, utf8Slice("20180512"))), + ImmutableMap.of( + "city", multipleValues(VARCHAR, ImmutableList.of(utf8Slice("sh"), utf8Slice("sz"))), + "date", singleValue(VARCHAR, utf8Slice("20180512")))); + } + + @Test + public void filterOnPartitionColumnAndRegularColumns() + { + String tableName = "deltatbl-partition-prune"; + String testQuery = format("SELECT date, name, city, cnt FROM \"%s\" WHERE city in ('sh', 'sz') AND name = 'Linda'", tableName); + String expResultsQuery = "SELECT * FROM VALUES('20181212', 'Linda', 'sz', 8)"; + + assertDeltaQueryOptimized( + tableName, + testQuery, + expResultsQuery, + ImmutableMap.of( + "city", multipleValues(VARCHAR, ImmutableList.of(utf8Slice("sh"), utf8Slice("sz"))), + "name", singleValue(VARCHAR, utf8Slice("Linda"))), + ImmutableMap.of("city", multipleValues(VARCHAR, ImmutableList.of(utf8Slice("sh"), utf8Slice("sz"))))); + } + + @Test + public void nullPartitionFilter() + { + String tableName = "data-reader-partition-values"; + String testQuery = + format("SELECT value, as_boolean FROM \"%s\" WHERE as_int is null and value is not null", tableName); + String expResultsQuery = "SELECT * FROM VALUES('2', null)"; + + assertDeltaQueryOptimized( + tableName, + testQuery, + expResultsQuery, + ImmutableMap.of( + "as_int", onlyNull(INTEGER), + "value", notNull(VARCHAR)), + ImmutableMap.of("as_int", onlyNull(INTEGER))); + } + + @Test + public void notNullPartitionFilter() + { + String tableName = "data-reader-partition-values"; + String testQuery = format("SELECT value, as_boolean FROM \"%s\" WHERE as_int is not null and value = '1'", tableName); + String expResultsQuery = "SELECT * FROM VALUES('1', false)"; + + assertDeltaQueryOptimized( + tableName, + testQuery, + expResultsQuery, + ImmutableMap.of( + "as_int", notNull(INTEGER), + "value", singleValue(VARCHAR, utf8Slice("1"))), + ImmutableMap.of("as_int", notNull(INTEGER))); + } + + @Test + public void nestedColumnFilter() + { + String tableName = "data-reader-nested-struct"; + String testQuery = format("SELECT a.aa, a.ac.aca FROM \"%s\" WHERE a.aa in ('8', '9') AND a.ac.aca > 6", tableName); + String expResultsQuery = "SELECT * FROM VALUES('8', 8),('9', 9)"; + + assertDeltaQueryOptimized( + tableName, + testQuery, + expResultsQuery, + ImmutableMap.of( + "a$_$_$aa", multipleValues(VARCHAR, ImmutableList.of(utf8Slice("8"), utf8Slice("9"))), + "a$_$_$ac$_$_$aca", Domain.create( + SortedRangeSet.copyOf( + INTEGER, + ImmutableList.of(Range.greaterThan(INTEGER, 6L))), + false)), + ImmutableMap.of()); + } + + protected void assertDeltaQueryOptimized( + String tableName, + String testQuery, + String expResultsQuery, + Map expectedConstraint, + Map expectedEnforcedConstraint) + { + try { + registerDeltaTableInHMS(tableName, tableName); + + // verify the query returns correct results + assertQuery(testQuery, expResultsQuery); + + // verify the plan contains filter pushed down into scan appropriately + transaction(getQueryRunner().getTransactionManager(), getQueryRunner().getAccessControl()) + .execute(getSession(), session -> { + Plan plan = getQueryRunner().createPlan(session, testQuery, WarningCollector.NOOP); + assertPlan( + session, + getQueryRunner().getMetadata(), + (node, sourceStats, lookup, ignore, types) -> PlanNodeStatsEstimate.unknown(), + plan, + anyTree(tableScanWithConstraints( + tableName, + expectedConstraint, + expectedEnforcedConstraint))); + }); + } + finally { + unregisterDeltaTableInHMS(tableName); + } + } + + /** + * Utility plan verification method that checks whether the table scan node has given constraint. + */ + private static PlanMatchPattern tableScanWithConstraints( + String tableName, + Map expectedConstraint, + Map expectedEnforcedConstraint) + { + return PlanMatchPattern.tableScan(tableName).with(new Matcher() + { + @Override + public boolean shapeMatches(PlanNode node) + { + return node instanceof TableScanNode; + } + + @Override + public MatchResult detailMatches(PlanNode node, StatsProvider stats, Session session, Metadata metadata, SymbolAliases symbolAliases) + { + TableScanNode tableScan = (TableScanNode) node; + Map actualEnforcedConstraint = tableScan.getEnforcedConstraint() + .transform(DeltaColumnHandle.class::cast) + .transform(DeltaColumnHandle::getName) + .getDomains() + .get(); + DeltaTableHandle deltaTableHandle = (DeltaTableHandle) tableScan.getTable().getConnectorHandle(); + Map actualConstraint = deltaTableHandle.getPredicate() + .transform(DeltaColumnHandle::getName) + .getDomains() + .get(); + + if (!expectedConstraint.equals(actualConstraint) || !expectedEnforcedConstraint.equals(actualEnforcedConstraint)) { + return NO_MATCH; + } + + return match(); + } + }); + } +} diff --git a/presto-delta/src/test/java/io/prestosql/delta/TestDeltaSplit.java b/presto-delta/src/test/java/io/prestosql/delta/TestDeltaSplit.java new file mode 100644 index 000000000000..f2a7ea85d48c --- /dev/null +++ b/presto-delta/src/test/java/io/prestosql/delta/TestDeltaSplit.java @@ -0,0 +1,56 @@ +/* + * 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.prestosql.delta; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.airlift.json.JsonCodec; +import org.testng.annotations.Test; + +import static org.testng.Assert.assertEquals; + +/** + * Test {@link DeltaSplit} is created correctly with given arguments and JSON serialization/deserialization works. + */ +public class TestDeltaSplit +{ + private final JsonCodec codec = JsonCodec.jsonCodec(DeltaSplit.class); + + @Test + public void testJsonRoundTrip() + { + DeltaSplit expected = new DeltaSplit( + "delta", + "database", + "table", + "s3://bucket/path/to/delta/table/file1.parquet", + 0, + 200, + 500, + ImmutableMap.of("part1", "part1Val")); + + String json = codec.toJson(expected); + DeltaSplit actual = codec.fromJson(json); + + assertEquals(actual.getConnectorId(), expected.getConnectorId()); + assertEquals(actual.getSchema(), expected.getSchema()); + assertEquals(actual.getTable(), expected.getTable()); + assertEquals(actual.getFilePath(), expected.getFilePath()); + assertEquals(actual.getStart(), expected.getStart()); + assertEquals(actual.getLength(), expected.getLength()); + assertEquals(actual.getFileSize(), expected.getFileSize()); + assertEquals(actual.getAddresses(), ImmutableList.of()); + assertEquals(actual.getPartitionValues(), expected.getPartitionValues()); + } +} diff --git a/presto-delta/src/test/java/io/prestosql/delta/TestDeltaTableHandle.java b/presto-delta/src/test/java/io/prestosql/delta/TestDeltaTableHandle.java new file mode 100644 index 000000000000..67aed9061239 --- /dev/null +++ b/presto-delta/src/test/java/io/prestosql/delta/TestDeltaTableHandle.java @@ -0,0 +1,130 @@ +/* + * 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.prestosql.delta; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import io.airlift.bootstrap.Bootstrap; +import io.airlift.json.JsonBinder; +import io.airlift.json.JsonCodec; +import io.airlift.json.JsonModule; +import io.prestosql.block.BlockJsonSerde; +import io.prestosql.metadata.HandleJsonModule; +import io.prestosql.metadata.HandleResolver; +import io.prestosql.spi.block.Block; +import io.prestosql.spi.block.TestingBlockEncodingSerde; +import io.prestosql.spi.predicate.Domain; +import io.prestosql.spi.predicate.Range; +import io.prestosql.spi.predicate.SortedRangeSet; +import io.prestosql.spi.predicate.TupleDomain; +import io.prestosql.spi.type.TestingTypeDeserializer; +import io.prestosql.spi.type.TestingTypeManager; +import io.prestosql.spi.type.Type; +import io.prestosql.spi.type.TypeSignature; +import io.prestosql.type.TypeSignatureDeserializer; +import org.testng.annotations.Test; + +import java.util.List; +import java.util.Optional; + +import static io.airlift.json.JsonBinder.jsonBinder; +import static io.airlift.json.JsonCodecBinder.jsonCodecBinder; +import static io.prestosql.delta.DeltaColumnHandle.ColumnType.PARTITION; +import static io.prestosql.delta.DeltaColumnHandle.ColumnType.REGULAR; +import static io.prestosql.spi.type.DateType.DATE; +import static io.prestosql.spi.type.DoubleType.DOUBLE; +import static io.prestosql.spi.type.IntegerType.INTEGER; +import static io.prestosql.spi.type.RealType.REAL; +import static org.testng.Assert.assertEquals; + +/** + * Test {@link DeltaTableHandle} is created correctly with given arguments and JSON serialization/deserialization works. + */ +public class TestDeltaTableHandle +{ + @Test + public void testJsonRoundTrip() + { + List columns = ImmutableList.of( + new DeltaColumn("c1", REAL.getTypeSignature(), true, true), + new DeltaColumn("c2", INTEGER.getTypeSignature(), false, true), + new DeltaColumn("c3", DOUBLE.getTypeSignature(), false, false), + new DeltaColumn("c4", DATE.getTypeSignature(), true, false)); + + DeltaTable deltaTable = new DeltaTable( + "schema", + "table", + "s3:/bucket/table/location", + Optional.of(1L), + columns); + + DeltaColumnHandle c3ColumnHandle = new DeltaColumnHandle( + "delta", + columns.get(2).getName(), + columns.get(2).getType(), + columns.get(2).isPartition() ? PARTITION : REGULAR); + + TupleDomain predicate = TupleDomain.withColumnDomains(ImmutableMap.of( + c3ColumnHandle, Domain.create(SortedRangeSet.copyOf(DOUBLE, + ImmutableList.of( + Range.equal(DOUBLE, (double) (100.0f + 0)), + Range.equal(DOUBLE, (double) (100.008f + 0)), + Range.equal(DOUBLE, (double) (100.0f + 14)))), + false))); + + DeltaTableHandle expected = new DeltaTableHandle( + "delta", + deltaTable, + predicate, + Optional.of("predicateString")); + + String json = getJsonCodec().toJson(expected); + DeltaTableHandle actual = getJsonCodec().fromJson(json); + + assertEquals(actual.getDeltaTable(), expected.getDeltaTable()); + assertEquals(actual.getPredicate(), expected.getPredicate()); + assertEquals(actual.getPredicateString(), expected.getPredicateString()); + } + + private JsonCodec getJsonCodec() + { + Module module = binder -> { + binder.install(new JsonModule()); + binder.install(new HandleJsonModule()); + + TestingBlockEncodingSerde blockEncodingSerde = new TestingBlockEncodingSerde(); + + JsonBinder jsonBinder = jsonBinder(binder); + jsonBinder.addDeserializerBinding(Type.class).toInstance(new TestingTypeDeserializer(new TestingTypeManager())); + jsonBinder.addSerializerBinding(Block.class).toInstance(new BlockJsonSerde.Serializer(blockEncodingSerde)); + jsonBinder.addDeserializerBinding(Block.class).toInstance(new BlockJsonSerde.Deserializer(blockEncodingSerde)); + jsonBinder.addDeserializerBinding(TypeSignature.class).to(TypeSignatureDeserializer.class); + + jsonCodecBinder(binder).bindJsonCodec(DeltaTableHandle.class); + }; + Bootstrap app = new Bootstrap(ImmutableList.of(module)); + Injector injector = app + .doNotInitializeLogging() + .quiet() + .initialize(); + HandleResolver handleResolver = injector.getInstance(HandleResolver.class); + handleResolver.addCatalogHandleResolver("delta", new DeltaConnectionHandleResolver()); + return injector.getInstance(new Key<>() + { + }); + } +} diff --git a/presto-delta/src/test/java/io/prestosql/delta/TestDeltaTableName.java b/presto-delta/src/test/java/io/prestosql/delta/TestDeltaTableName.java new file mode 100644 index 000000000000..12e19e268e02 --- /dev/null +++ b/presto-delta/src/test/java/io/prestosql/delta/TestDeltaTableName.java @@ -0,0 +1,95 @@ +/* + * 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.prestosql.delta; + +import io.prestosql.spi.PrestoException; +import org.testng.annotations.Test; + +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.util.Optional; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; + +public class TestDeltaTableName +{ + private static final DateTimeFormatter TIMESTAMP_PARSER = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss"); + + @Test + public void parsingValidTableNames() + { + assertValid("tbl", "tbl"); + assertValid("tbl@v2", "tbl", 2L); + assertValid("s3://bucket/path/tbl", "s3://bucket/path/tbl"); + assertValid("s3://bucket/path/tbl@v5", "s3://bucket/path/tbl", 5L); + assertValid("tbl@t2021", "tbl", "2021-01-01 00:00:00"); + assertValid("tbl@t2021-11", "tbl", "2021-11-01 00:00:00"); + assertValid("tbl@t2021-11-18", "tbl", "2021-11-18 00:00:00"); + assertValid("tbl@t2021-11-18 10", "tbl", "2021-11-18 10:00:00"); + assertValid("tbl@t2021-11-18 10:30", "tbl", "2021-11-18 10:30:00"); + assertValid("tbl@t2021-11-18 10:30:03", "tbl", "2021-11-18 10:30:03"); + } + + @Test + public void parsingInvalidTableNames() + { + assertInvalid("sales_table@v-1", + "Invalid Delta table name: sales_table@v-1, " + + "Expected table name form 'tableName[@v][@t]'."); + + assertInvalid("sales_table@t2021-14", + "Invalid Delta table name: sales_table@t2021-14, given snapshot timestamp (2021-14) format is not valid"); + } + + private static void assertValid(String inputTableName, String expectedTableName) + { + DeltaTableName deltaTableName = DeltaTableName.from(inputTableName); + assertEquals(deltaTableName.getTableNameOrPath(), expectedTableName); + assertEquals(deltaTableName.getSnapshotId(), Optional.empty()); + assertEquals(deltaTableName.getSnapshotAsOfTimestamp(), Optional.empty()); + } + + private static void assertValid(String inputTableName, String expectedTableName, Long expectedSnapshotId) + { + DeltaTableName deltaTableName = DeltaTableName.from(inputTableName); + assertEquals(deltaTableName.getTableNameOrPath(), expectedTableName); + assertEquals(deltaTableName.getSnapshotId(), Optional.of(expectedSnapshotId)); + assertEquals(deltaTableName.getSnapshotAsOfTimestamp(), Optional.empty()); + } + + private static void assertValid(String inputTableName, String expectedTableName, String expectedTimestamp) + { + DeltaTableName deltaTableName = DeltaTableName.from(inputTableName); + assertEquals(deltaTableName.getTableNameOrPath(), expectedTableName); + assertEquals(deltaTableName.getSnapshotId(), Optional.empty()); + Long expectedTimestampEpochMillis = LocalDateTime.from(TIMESTAMP_PARSER.parse(expectedTimestamp)) + .toInstant(ZoneOffset.UTC) + .toEpochMilli(); + assertEquals(deltaTableName.getSnapshotAsOfTimestamp(), Optional.of(expectedTimestampEpochMillis)); + } + + private static void assertInvalid(String inputTableName, String expErrorMessage) + { + try { + DeltaTableName.from(inputTableName); + fail("expected the above call to fail"); + } + catch (PrestoException ex) { + assertTrue(ex.getMessage().contains(expErrorMessage), ex.getMessage()); + } + } +} diff --git a/presto-delta/src/test/java/io/prestosql/delta/TestDeltaTypeUtils.java b/presto-delta/src/test/java/io/prestosql/delta/TestDeltaTypeUtils.java new file mode 100644 index 000000000000..7c318d62ac83 --- /dev/null +++ b/presto-delta/src/test/java/io/prestosql/delta/TestDeltaTypeUtils.java @@ -0,0 +1,82 @@ +/* + * 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.prestosql.delta; + +import io.airlift.slice.Slices; +import io.prestosql.spi.PrestoException; +import io.prestosql.spi.type.Type; +import org.testng.annotations.Test; + +import java.math.BigInteger; + +import static io.prestosql.delta.DeltaErrorCode.DELTA_INVALID_PARTITION_VALUE; +import static io.prestosql.delta.DeltaTypeUtils.convertPartitionValue; +import static io.prestosql.spi.type.BigintType.BIGINT; +import static io.prestosql.spi.type.BooleanType.BOOLEAN; +import static io.prestosql.spi.type.DateType.DATE; +import static io.prestosql.spi.type.DecimalType.createDecimalType; +import static io.prestosql.spi.type.Decimals.encodeUnscaledValue; +import static io.prestosql.spi.type.DoubleType.DOUBLE; +import static io.prestosql.spi.type.IntegerType.INTEGER; +import static io.prestosql.spi.type.RealType.REAL; +import static io.prestosql.spi.type.SmallintType.SMALLINT; +import static io.prestosql.spi.type.TimestampType.TIMESTAMP; +import static io.prestosql.spi.type.TinyintType.TINYINT; +import static io.prestosql.spi.type.VarcharType.VARCHAR; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; + +public class TestDeltaTypeUtils +{ + @Test + public void partitionValueParsing() + { + assertPartitionValue("str", VARCHAR, Slices.utf8Slice("str")); + assertPartitionValue("3", TINYINT, 3L); + assertPartitionValue("344", SMALLINT, 344L); + assertPartitionValue("323243", INTEGER, 323243L); + assertPartitionValue("234234234233", BIGINT, 234234234233L); + assertPartitionValue("3.234234", REAL, 1078918577L); + assertPartitionValue("34534.23423423423", DOUBLE, 34534.23423423423); + assertPartitionValue("2021-11-18", DATE, 18949L); + assertPartitionValue("2021-11-18 05:23:43", TIMESTAMP, 1637213023000L); + assertPartitionValue("true", BOOLEAN, true); + assertPartitionValue("faLse", BOOLEAN, false); + assertPartitionValue("234.5", createDecimalType(6, 3), 2345L); + assertPartitionValue("12345678901234567890123.5", createDecimalType(23, 1), encodeUnscaledValue(new BigInteger("123456789012345678901235"))); + + invalidPartitionValue("sdfsdf", BOOLEAN); + invalidPartitionValue("sdfsdf", DATE); + invalidPartitionValue("sdfsdf", TIMESTAMP); + } + + private void assertPartitionValue(String value, Type type, Object expected) + { + Object actual = convertPartitionValue("p1", value, type.getTypeSignature()); + assertEquals(actual, expected); + } + + private void invalidPartitionValue(String value, Type type) + { + try { + convertPartitionValue("p1", value, type.getTypeSignature()); + fail("expected to fail"); + } + catch (PrestoException e) { + assertEquals(e.getErrorCode(), DELTA_INVALID_PARTITION_VALUE.toErrorCode()); + assertTrue(e.getMessage().matches("Can not parse partition value .* of type .* for partition column 'p1'")); + } + } +} diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-106da899-cbd5-491f-b560-60792edbe807-c000.snappy.parquet.crc b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-106da899-cbd5-491f-b560-60792edbe807-c000.snappy.parquet.crc new file mode 100644 index 000000000000..9e1378be7722 Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-106da899-cbd5-491f-b560-60792edbe807-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-12ef2b67-494f-4eff-8e56-2431d0adf4bb-c000.snappy.parquet.crc b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-12ef2b67-494f-4eff-8e56-2431d0adf4bb-c000.snappy.parquet.crc new file mode 100644 index 000000000000..34a63575cea7 Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-12ef2b67-494f-4eff-8e56-2431d0adf4bb-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-197ae400-931d-4eb2-8059-a4688c2d59a9-c000.snappy.parquet.crc b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-197ae400-931d-4eb2-8059-a4688c2d59a9-c000.snappy.parquet.crc new file mode 100644 index 000000000000..4b6155c66938 Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-197ae400-931d-4eb2-8059-a4688c2d59a9-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-27b57b04-e80f-48b2-9064-c459e9d26a87-c000.snappy.parquet.crc b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-27b57b04-e80f-48b2-9064-c459e9d26a87-c000.snappy.parquet.crc new file mode 100644 index 000000000000..f8ab1bd81a17 Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-27b57b04-e80f-48b2-9064-c459e9d26a87-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-2e12af63-21db-42d7-9bda-766b1255e6f6-c000.snappy.parquet.crc b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-2e12af63-21db-42d7-9bda-766b1255e6f6-c000.snappy.parquet.crc new file mode 100644 index 000000000000..35c6c50fe42b Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-2e12af63-21db-42d7-9bda-766b1255e6f6-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-3a090e55-ad26-4e2e-8991-5fc96e11200a-c000.snappy.parquet.crc b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-3a090e55-ad26-4e2e-8991-5fc96e11200a-c000.snappy.parquet.crc new file mode 100644 index 000000000000..1a7961289206 Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-3a090e55-ad26-4e2e-8991-5fc96e11200a-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-4a9efe8d-12b3-4134-a932-a4e288cadf35-c000.snappy.parquet.crc b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-4a9efe8d-12b3-4134-a932-a4e288cadf35-c000.snappy.parquet.crc new file mode 100644 index 000000000000..8a106081008b Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-4a9efe8d-12b3-4134-a932-a4e288cadf35-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-4aa85334-6e99-4dac-9d75-dfd59ec37250-c000.snappy.parquet.crc b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-4aa85334-6e99-4dac-9d75-dfd59ec37250-c000.snappy.parquet.crc new file mode 100644 index 000000000000..ff43b12f7ac7 Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-4aa85334-6e99-4dac-9d75-dfd59ec37250-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-4fb7f274-cd1b-484d-95d3-42521e10e10d-c000.snappy.parquet.crc b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-4fb7f274-cd1b-484d-95d3-42521e10e10d-c000.snappy.parquet.crc new file mode 100644 index 000000000000..10dd26caf322 Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-4fb7f274-cd1b-484d-95d3-42521e10e10d-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-5d1ee207-355b-47ee-b58d-5cf32a3b8349-c000.snappy.parquet.crc b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-5d1ee207-355b-47ee-b58d-5cf32a3b8349-c000.snappy.parquet.crc new file mode 100644 index 000000000000..86838de0d3cc Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-5d1ee207-355b-47ee-b58d-5cf32a3b8349-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-7f8faf7f-2d0f-42c8-84c9-85df909dba39-c000.snappy.parquet.crc b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-7f8faf7f-2d0f-42c8-84c9-85df909dba39-c000.snappy.parquet.crc new file mode 100644 index 000000000000..fb29ac6d4e76 Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-7f8faf7f-2d0f-42c8-84c9-85df909dba39-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-999ae312-021c-4823-b0b6-08014cea263a-c000.snappy.parquet.crc b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-999ae312-021c-4823-b0b6-08014cea263a-c000.snappy.parquet.crc new file mode 100644 index 000000000000..6976dff400e3 Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-999ae312-021c-4823-b0b6-08014cea263a-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-9c2ae306-eeca-4e27-af5c-88e4536e5bd3-c000.snappy.parquet.crc b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-9c2ae306-eeca-4e27-af5c-88e4536e5bd3-c000.snappy.parquet.crc new file mode 100644 index 000000000000..9f40fa723d33 Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-9c2ae306-eeca-4e27-af5c-88e4536e5bd3-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-a12a3c66-8abb-40a8-8d69-bbf7e516b710-c000.snappy.parquet.crc b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-a12a3c66-8abb-40a8-8d69-bbf7e516b710-c000.snappy.parquet.crc new file mode 100644 index 000000000000..6fb286b647fb Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-a12a3c66-8abb-40a8-8d69-bbf7e516b710-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-a90c681e-2310-4bbc-8b56-e81b14aa1817-c000.snappy.parquet.crc b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-a90c681e-2310-4bbc-8b56-e81b14aa1817-c000.snappy.parquet.crc new file mode 100644 index 000000000000..4052974e23ec Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-a90c681e-2310-4bbc-8b56-e81b14aa1817-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-bf15b1cf-2d02-47fd-aa12-1117de3f9071-c000.snappy.parquet.crc b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-bf15b1cf-2d02-47fd-aa12-1117de3f9071-c000.snappy.parquet.crc new file mode 100644 index 000000000000..e27c2130040b Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-bf15b1cf-2d02-47fd-aa12-1117de3f9071-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-df985307-35c6-4c43-9635-64f6e9ce9e3b-c000.snappy.parquet.crc b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-df985307-35c6-4c43-9635-64f6e9ce9e3b-c000.snappy.parquet.crc new file mode 100644 index 000000000000..40ed3a37c1e6 Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-df985307-35c6-4c43-9635-64f6e9ce9e3b-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-e048b59b-38c6-417a-b452-41539eadc475-c000.snappy.parquet.crc b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-e048b59b-38c6-417a-b452-41539eadc475-c000.snappy.parquet.crc new file mode 100644 index 000000000000..71f6b68fcf7f Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-e048b59b-38c6-417a-b452-41539eadc475-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-e22924ef-f192-483a-9b57-82d2123ba08e-c000.snappy.parquet.crc b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-e22924ef-f192-483a-9b57-82d2123ba08e-c000.snappy.parquet.crc new file mode 100644 index 000000000000..c36454b7fe8e Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-e22924ef-f192-483a-9b57-82d2123ba08e-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-e877c5f2-5fb9-4528-a317-5b30608bb45c-c000.snappy.parquet.crc b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-e877c5f2-5fb9-4528-a317-5b30608bb45c-c000.snappy.parquet.crc new file mode 100644 index 000000000000..9dcd9876ad9d Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/.part-00000-e877c5f2-5fb9-4528-a317-5b30608bb45c-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/.00000000000000000010.checkpoint.parquet.crc b/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/.00000000000000000010.checkpoint.parquet.crc new file mode 100644 index 000000000000..b7d844826a41 Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/.00000000000000000010.checkpoint.parquet.crc differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000010.checkpoint.parquet b/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000010.checkpoint.parquet new file mode 100644 index 000000000000..46a216cea9da Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000010.checkpoint.parquet differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000011.json b/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000011.json new file mode 100644 index 000000000000..28df92900111 --- /dev/null +++ b/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000011.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1636316599141,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":10,"isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputBytes":"687","numOutputRows":"1"}}} +{"add":{"path":"part-00000-5d1ee207-355b-47ee-b58d-5cf32a3b8349-c000.snappy.parquet","partitionValues":{},"size":687,"modificationTime":1636316599000,"dataChange":true}} diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000012.json b/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000012.json new file mode 100644 index 000000000000..fc3f3f1b381b --- /dev/null +++ b/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000012.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1636316599964,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":11,"isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputBytes":"687","numOutputRows":"1"}}} +{"add":{"path":"part-00000-4aa85334-6e99-4dac-9d75-dfd59ec37250-c000.snappy.parquet","partitionValues":{},"size":687,"modificationTime":1636316599000,"dataChange":true}} diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000013.json b/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000013.json new file mode 100644 index 000000000000..a373531343b2 --- /dev/null +++ b/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000013.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1636316600652,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":12,"isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputBytes":"687","numOutputRows":"1"}}} +{"add":{"path":"part-00000-e22924ef-f192-483a-9b57-82d2123ba08e-c000.snappy.parquet","partitionValues":{},"size":687,"modificationTime":1636316600000,"dataChange":true}} diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000014.json b/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000014.json new file mode 100644 index 000000000000..8fce899ec58c --- /dev/null +++ b/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000014.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1636316601328,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":13,"isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputBytes":"687","numOutputRows":"1"}}} +{"add":{"path":"part-00000-3a090e55-ad26-4e2e-8991-5fc96e11200a-c000.snappy.parquet","partitionValues":{},"size":687,"modificationTime":1636316601000,"dataChange":true}} diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000015.json b/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000015.json new file mode 100644 index 000000000000..9abe0952e0c7 --- /dev/null +++ b/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000015.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1636316601978,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":14,"isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputBytes":"687","numOutputRows":"1"}}} +{"add":{"path":"part-00000-e877c5f2-5fb9-4528-a317-5b30608bb45c-c000.snappy.parquet","partitionValues":{},"size":687,"modificationTime":1636316601000,"dataChange":true}} diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000016.json b/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000016.json new file mode 100644 index 000000000000..886088f6884c --- /dev/null +++ b/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000016.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1636316602640,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":15,"isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputBytes":"687","numOutputRows":"1"}}} +{"add":{"path":"part-00000-27b57b04-e80f-48b2-9064-c459e9d26a87-c000.snappy.parquet","partitionValues":{},"size":687,"modificationTime":1636316602000,"dataChange":true}} diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000017.json b/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000017.json new file mode 100644 index 000000000000..a2902a410601 --- /dev/null +++ b/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000017.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1636316603292,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":16,"isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputBytes":"687","numOutputRows":"1"}}} +{"add":{"path":"part-00000-2e12af63-21db-42d7-9bda-766b1255e6f6-c000.snappy.parquet","partitionValues":{},"size":687,"modificationTime":1636316603000,"dataChange":true}} diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000018.json b/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000018.json new file mode 100644 index 000000000000..e50020a04317 --- /dev/null +++ b/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000018.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1636316603965,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":17,"isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputBytes":"687","numOutputRows":"1"}}} +{"add":{"path":"part-00000-7f8faf7f-2d0f-42c8-84c9-85df909dba39-c000.snappy.parquet","partitionValues":{},"size":687,"modificationTime":1636316603000,"dataChange":true}} diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000019.json b/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000019.json new file mode 100644 index 000000000000..5e8d7dfdae44 --- /dev/null +++ b/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/00000000000000000019.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1636316604635,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":18,"isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputBytes":"687","numOutputRows":"1"}}} +{"add":{"path":"part-00000-df985307-35c6-4c43-9635-64f6e9ce9e3b-c000.snappy.parquet","partitionValues":{},"size":687,"modificationTime":1636316604000,"dataChange":true}} diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/_last_checkpoint b/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/_last_checkpoint new file mode 100644 index 000000000000..e125139624d2 --- /dev/null +++ b/presto-delta/src/test/resources/checkpointed-delta-table/_delta_log/_last_checkpoint @@ -0,0 +1 @@ +{"version":10,"size":13} diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-106da899-cbd5-491f-b560-60792edbe807-c000.snappy.parquet b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-106da899-cbd5-491f-b560-60792edbe807-c000.snappy.parquet new file mode 100644 index 000000000000..aa8eef07d4e2 Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-106da899-cbd5-491f-b560-60792edbe807-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-12ef2b67-494f-4eff-8e56-2431d0adf4bb-c000.snappy.parquet b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-12ef2b67-494f-4eff-8e56-2431d0adf4bb-c000.snappy.parquet new file mode 100644 index 000000000000..c4b563d4d7ff Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-12ef2b67-494f-4eff-8e56-2431d0adf4bb-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-197ae400-931d-4eb2-8059-a4688c2d59a9-c000.snappy.parquet b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-197ae400-931d-4eb2-8059-a4688c2d59a9-c000.snappy.parquet new file mode 100644 index 000000000000..8ece6b1c7c6a Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-197ae400-931d-4eb2-8059-a4688c2d59a9-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-27b57b04-e80f-48b2-9064-c459e9d26a87-c000.snappy.parquet b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-27b57b04-e80f-48b2-9064-c459e9d26a87-c000.snappy.parquet new file mode 100644 index 000000000000..a0bc805945e6 Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-27b57b04-e80f-48b2-9064-c459e9d26a87-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-2e12af63-21db-42d7-9bda-766b1255e6f6-c000.snappy.parquet b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-2e12af63-21db-42d7-9bda-766b1255e6f6-c000.snappy.parquet new file mode 100644 index 000000000000..d79953afa7db Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-2e12af63-21db-42d7-9bda-766b1255e6f6-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-3a090e55-ad26-4e2e-8991-5fc96e11200a-c000.snappy.parquet b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-3a090e55-ad26-4e2e-8991-5fc96e11200a-c000.snappy.parquet new file mode 100644 index 000000000000..bf6353490ee8 Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-3a090e55-ad26-4e2e-8991-5fc96e11200a-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-4a9efe8d-12b3-4134-a932-a4e288cadf35-c000.snappy.parquet b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-4a9efe8d-12b3-4134-a932-a4e288cadf35-c000.snappy.parquet new file mode 100644 index 000000000000..90d1ce6d3f23 Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-4a9efe8d-12b3-4134-a932-a4e288cadf35-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-4aa85334-6e99-4dac-9d75-dfd59ec37250-c000.snappy.parquet b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-4aa85334-6e99-4dac-9d75-dfd59ec37250-c000.snappy.parquet new file mode 100644 index 000000000000..368bb75bed2e Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-4aa85334-6e99-4dac-9d75-dfd59ec37250-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-4fb7f274-cd1b-484d-95d3-42521e10e10d-c000.snappy.parquet b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-4fb7f274-cd1b-484d-95d3-42521e10e10d-c000.snappy.parquet new file mode 100644 index 000000000000..a826957602af Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-4fb7f274-cd1b-484d-95d3-42521e10e10d-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-5d1ee207-355b-47ee-b58d-5cf32a3b8349-c000.snappy.parquet b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-5d1ee207-355b-47ee-b58d-5cf32a3b8349-c000.snappy.parquet new file mode 100644 index 000000000000..dd70fbff5ebb Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-5d1ee207-355b-47ee-b58d-5cf32a3b8349-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-7f8faf7f-2d0f-42c8-84c9-85df909dba39-c000.snappy.parquet b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-7f8faf7f-2d0f-42c8-84c9-85df909dba39-c000.snappy.parquet new file mode 100644 index 000000000000..4968c8854b3e Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-7f8faf7f-2d0f-42c8-84c9-85df909dba39-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-999ae312-021c-4823-b0b6-08014cea263a-c000.snappy.parquet b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-999ae312-021c-4823-b0b6-08014cea263a-c000.snappy.parquet new file mode 100644 index 000000000000..f397656c450d Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-999ae312-021c-4823-b0b6-08014cea263a-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-9c2ae306-eeca-4e27-af5c-88e4536e5bd3-c000.snappy.parquet b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-9c2ae306-eeca-4e27-af5c-88e4536e5bd3-c000.snappy.parquet new file mode 100644 index 000000000000..7d73f50e2b75 Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-9c2ae306-eeca-4e27-af5c-88e4536e5bd3-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-a12a3c66-8abb-40a8-8d69-bbf7e516b710-c000.snappy.parquet b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-a12a3c66-8abb-40a8-8d69-bbf7e516b710-c000.snappy.parquet new file mode 100644 index 000000000000..bcb024458bb3 Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-a12a3c66-8abb-40a8-8d69-bbf7e516b710-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-a90c681e-2310-4bbc-8b56-e81b14aa1817-c000.snappy.parquet b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-a90c681e-2310-4bbc-8b56-e81b14aa1817-c000.snappy.parquet new file mode 100644 index 000000000000..691fb0e58f85 Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-a90c681e-2310-4bbc-8b56-e81b14aa1817-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-bf15b1cf-2d02-47fd-aa12-1117de3f9071-c000.snappy.parquet b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-bf15b1cf-2d02-47fd-aa12-1117de3f9071-c000.snappy.parquet new file mode 100644 index 000000000000..df73e724ab37 Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-bf15b1cf-2d02-47fd-aa12-1117de3f9071-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-df985307-35c6-4c43-9635-64f6e9ce9e3b-c000.snappy.parquet b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-df985307-35c6-4c43-9635-64f6e9ce9e3b-c000.snappy.parquet new file mode 100644 index 000000000000..0f5cab0ef1b1 Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-df985307-35c6-4c43-9635-64f6e9ce9e3b-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-e048b59b-38c6-417a-b452-41539eadc475-c000.snappy.parquet b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-e048b59b-38c6-417a-b452-41539eadc475-c000.snappy.parquet new file mode 100644 index 000000000000..b94d46f52448 Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-e048b59b-38c6-417a-b452-41539eadc475-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-e22924ef-f192-483a-9b57-82d2123ba08e-c000.snappy.parquet b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-e22924ef-f192-483a-9b57-82d2123ba08e-c000.snappy.parquet new file mode 100644 index 000000000000..c752e4280d5c Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-e22924ef-f192-483a-9b57-82d2123ba08e-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-e877c5f2-5fb9-4528-a317-5b30608bb45c-c000.snappy.parquet b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-e877c5f2-5fb9-4528-a317-5b30608bb45c-c000.snappy.parquet new file mode 100644 index 000000000000..9ab06ab743e0 Binary files /dev/null and b/presto-delta/src/test/resources/checkpointed-delta-table/part-00000-e877c5f2-5fb9-4528-a317-5b30608bb45c-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/data-reader-array-complex-objects/.part-00000-a7d58b1a-7743-4bb0-b208-438bbe179c93-c000.snappy.parquet.crc b/presto-delta/src/test/resources/data-reader-array-complex-objects/.part-00000-a7d58b1a-7743-4bb0-b208-438bbe179c93-c000.snappy.parquet.crc new file mode 100644 index 000000000000..2950d858b40a Binary files /dev/null and b/presto-delta/src/test/resources/data-reader-array-complex-objects/.part-00000-a7d58b1a-7743-4bb0-b208-438bbe179c93-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/data-reader-array-complex-objects/.part-00001-7b211746-0a31-4e77-9822-b0985158cd66-c000.snappy.parquet.crc b/presto-delta/src/test/resources/data-reader-array-complex-objects/.part-00001-7b211746-0a31-4e77-9822-b0985158cd66-c000.snappy.parquet.crc new file mode 100644 index 000000000000..d52f585dc0de Binary files /dev/null and b/presto-delta/src/test/resources/data-reader-array-complex-objects/.part-00001-7b211746-0a31-4e77-9822-b0985158cd66-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/data-reader-array-complex-objects/_delta_log/00000000000000000000.json b/presto-delta/src/test/resources/data-reader-array-complex-objects/_delta_log/00000000000000000000.json new file mode 100644 index 000000000000..f859e2df129f --- /dev/null +++ b/presto-delta/src/test/resources/data-reader-array-complex-objects/_delta_log/00000000000000000000.json @@ -0,0 +1,5 @@ +{"commitInfo":{"timestamp":1603724039052,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[]"},"isBlindAppend":true}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"38be1738-32ad-448f-9e29-912a7536d4ca","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"i\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"3d_int_list\",\"type\":{\"type\":\"array\",\"elementType\":{\"type\":\"array\",\"elementType\":{\"type\":\"array\",\"elementType\":\"integer\",\"containsNull\":true},\"containsNull\":true},\"containsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"4d_int_list\",\"type\":{\"type\":\"array\",\"elementType\":{\"type\":\"array\",\"elementType\":{\"type\":\"array\",\"elementType\":{\"type\":\"array\",\"elementType\":\"integer\",\"containsNull\":true},\"containsNull\":true},\"containsNull\":true},\"containsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"list_of_maps\",\"type\":{\"type\":\"array\",\"elementType\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"long\",\"valueContainsNull\":true},\"containsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"list_of_records\",\"type\":{\"type\":\"array\",\"elementType\":{\"type\":\"struct\",\"fields\":[{\"name\":\"val\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]},\"containsNull\":true},\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1603724038935}} +{"add":{"path":"part-00000-a7d58b1a-7743-4bb0-b208-438bbe179c93-c000.snappy.parquet","partitionValues":{},"size":2830,"modificationTime":1603724039000,"dataChange":true}} +{"add":{"path":"part-00001-7b211746-0a31-4e77-9822-b0985158cd66-c000.snappy.parquet","partitionValues":{},"size":2832,"modificationTime":1603724039000,"dataChange":true}} diff --git a/presto-delta/src/test/resources/data-reader-array-complex-objects/part-00000-a7d58b1a-7743-4bb0-b208-438bbe179c93-c000.snappy.parquet b/presto-delta/src/test/resources/data-reader-array-complex-objects/part-00000-a7d58b1a-7743-4bb0-b208-438bbe179c93-c000.snappy.parquet new file mode 100644 index 000000000000..fa057de4dd61 Binary files /dev/null and b/presto-delta/src/test/resources/data-reader-array-complex-objects/part-00000-a7d58b1a-7743-4bb0-b208-438bbe179c93-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/data-reader-array-complex-objects/part-00001-7b211746-0a31-4e77-9822-b0985158cd66-c000.snappy.parquet b/presto-delta/src/test/resources/data-reader-array-complex-objects/part-00001-7b211746-0a31-4e77-9822-b0985158cd66-c000.snappy.parquet new file mode 100644 index 000000000000..e99ac42ae864 Binary files /dev/null and b/presto-delta/src/test/resources/data-reader-array-complex-objects/part-00001-7b211746-0a31-4e77-9822-b0985158cd66-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/data-reader-array-primitives/.part-00000-182665f0-30df-470d-a5cb-8d9d483ed390-c000.snappy.parquet.crc b/presto-delta/src/test/resources/data-reader-array-primitives/.part-00000-182665f0-30df-470d-a5cb-8d9d483ed390-c000.snappy.parquet.crc new file mode 100644 index 000000000000..003ddb680f82 Binary files /dev/null and b/presto-delta/src/test/resources/data-reader-array-primitives/.part-00000-182665f0-30df-470d-a5cb-8d9d483ed390-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/data-reader-array-primitives/.part-00001-2e274fe7-eb75-4b73-8c72-423ee747abc0-c000.snappy.parquet.crc b/presto-delta/src/test/resources/data-reader-array-primitives/.part-00001-2e274fe7-eb75-4b73-8c72-423ee747abc0-c000.snappy.parquet.crc new file mode 100644 index 000000000000..0890a315fa1f Binary files /dev/null and b/presto-delta/src/test/resources/data-reader-array-primitives/.part-00001-2e274fe7-eb75-4b73-8c72-423ee747abc0-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/data-reader-array-primitives/_delta_log/00000000000000000000.json b/presto-delta/src/test/resources/data-reader-array-primitives/_delta_log/00000000000000000000.json new file mode 100644 index 000000000000..4928f2bfa118 --- /dev/null +++ b/presto-delta/src/test/resources/data-reader-array-primitives/_delta_log/00000000000000000000.json @@ -0,0 +1,5 @@ +{"commitInfo":{"timestamp":1603724038064,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[]"},"isBlindAppend":true}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"caaa1362-3717-449b-ab9b-f7d8d536018d","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"as_array_int\",\"type\":{\"type\":\"array\",\"elementType\":\"integer\",\"containsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"as_array_long\",\"type\":{\"type\":\"array\",\"elementType\":\"long\",\"containsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"as_array_byte\",\"type\":{\"type\":\"array\",\"elementType\":\"byte\",\"containsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"as_array_short\",\"type\":{\"type\":\"array\",\"elementType\":\"short\",\"containsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"as_array_boolean\",\"type\":{\"type\":\"array\",\"elementType\":\"boolean\",\"containsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"as_array_float\",\"type\":{\"type\":\"array\",\"elementType\":\"float\",\"containsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"as_array_double\",\"type\":{\"type\":\"array\",\"elementType\":\"double\",\"containsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"as_array_string\",\"type\":{\"type\":\"array\",\"elementType\":\"string\",\"containsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"as_array_binary\",\"type\":{\"type\":\"array\",\"elementType\":\"binary\",\"containsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"as_array_big_decimal\",\"type\":{\"type\":\"array\",\"elementType\":\"decimal(1,0)\",\"containsNull\":true},\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1603724037970}} +{"add":{"path":"part-00000-182665f0-30df-470d-a5cb-8d9d483ed390-c000.snappy.parquet","partitionValues":{},"size":3627,"modificationTime":1603724038000,"dataChange":true}} +{"add":{"path":"part-00001-2e274fe7-eb75-4b73-8c72-423ee747abc0-c000.snappy.parquet","partitionValues":{},"size":3644,"modificationTime":1603724038000,"dataChange":true}} diff --git a/presto-delta/src/test/resources/data-reader-array-primitives/part-00000-182665f0-30df-470d-a5cb-8d9d483ed390-c000.snappy.parquet b/presto-delta/src/test/resources/data-reader-array-primitives/part-00000-182665f0-30df-470d-a5cb-8d9d483ed390-c000.snappy.parquet new file mode 100644 index 000000000000..d156253ed682 Binary files /dev/null and b/presto-delta/src/test/resources/data-reader-array-primitives/part-00000-182665f0-30df-470d-a5cb-8d9d483ed390-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/data-reader-array-primitives/part-00001-2e274fe7-eb75-4b73-8c72-423ee747abc0-c000.snappy.parquet b/presto-delta/src/test/resources/data-reader-array-primitives/part-00001-2e274fe7-eb75-4b73-8c72-423ee747abc0-c000.snappy.parquet new file mode 100644 index 000000000000..58faa50b7877 Binary files /dev/null and b/presto-delta/src/test/resources/data-reader-array-primitives/part-00001-2e274fe7-eb75-4b73-8c72-423ee747abc0-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/data-reader-map/.part-00000-d9004e55-077b-4728-9ee6-b3401faa46ba-c000.snappy.parquet.crc b/presto-delta/src/test/resources/data-reader-map/.part-00000-d9004e55-077b-4728-9ee6-b3401faa46ba-c000.snappy.parquet.crc new file mode 100644 index 000000000000..cb0434e0b3fc Binary files /dev/null and b/presto-delta/src/test/resources/data-reader-map/.part-00000-d9004e55-077b-4728-9ee6-b3401faa46ba-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/data-reader-map/.part-00001-3d30d085-4cde-471e-a396-12af34a70812-c000.snappy.parquet.crc b/presto-delta/src/test/resources/data-reader-map/.part-00001-3d30d085-4cde-471e-a396-12af34a70812-c000.snappy.parquet.crc new file mode 100644 index 000000000000..76407e6cdf96 Binary files /dev/null and b/presto-delta/src/test/resources/data-reader-map/.part-00001-3d30d085-4cde-471e-a396-12af34a70812-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/data-reader-map/_delta_log/00000000000000000000.json b/presto-delta/src/test/resources/data-reader-map/_delta_log/00000000000000000000.json new file mode 100644 index 000000000000..b885292b05dc --- /dev/null +++ b/presto-delta/src/test/resources/data-reader-map/_delta_log/00000000000000000000.json @@ -0,0 +1,5 @@ +{"commitInfo":{"timestamp":1603724039953,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[]"},"isBlindAppend":true}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"e52f2c3e-fac0-4b28-9627-2e33e6b85dc0","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"i\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"a\",\"type\":{\"type\":\"map\",\"keyType\":\"integer\",\"valueType\":\"integer\",\"valueContainsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"b\",\"type\":{\"type\":\"map\",\"keyType\":\"long\",\"valueType\":\"byte\",\"valueContainsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"c\",\"type\":{\"type\":\"map\",\"keyType\":\"short\",\"valueType\":\"boolean\",\"valueContainsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"d\",\"type\":{\"type\":\"map\",\"keyType\":\"float\",\"valueType\":\"double\",\"valueContainsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"e\",\"type\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"decimal(1,0)\",\"valueContainsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"f\",\"type\":{\"type\":\"map\",\"keyType\":\"integer\",\"valueType\":{\"type\":\"array\",\"elementType\":{\"type\":\"struct\",\"fields\":[{\"name\":\"val\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]},\"containsNull\":true},\"valueContainsNull\":true},\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1603724039866}} +{"add":{"path":"part-00000-d9004e55-077b-4728-9ee6-b3401faa46ba-c000.snappy.parquet","partitionValues":{},"size":3638,"modificationTime":1603724039000,"dataChange":true}} +{"add":{"path":"part-00001-3d30d085-4cde-471e-a396-12af34a70812-c000.snappy.parquet","partitionValues":{},"size":3655,"modificationTime":1603724039000,"dataChange":true}} diff --git a/presto-delta/src/test/resources/data-reader-map/part-00000-d9004e55-077b-4728-9ee6-b3401faa46ba-c000.snappy.parquet b/presto-delta/src/test/resources/data-reader-map/part-00000-d9004e55-077b-4728-9ee6-b3401faa46ba-c000.snappy.parquet new file mode 100644 index 000000000000..d4b401834c70 Binary files /dev/null and b/presto-delta/src/test/resources/data-reader-map/part-00000-d9004e55-077b-4728-9ee6-b3401faa46ba-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/data-reader-map/part-00001-3d30d085-4cde-471e-a396-12af34a70812-c000.snappy.parquet b/presto-delta/src/test/resources/data-reader-map/part-00001-3d30d085-4cde-471e-a396-12af34a70812-c000.snappy.parquet new file mode 100644 index 000000000000..421662a34e4a Binary files /dev/null and b/presto-delta/src/test/resources/data-reader-map/part-00001-3d30d085-4cde-471e-a396-12af34a70812-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/data-reader-nested-struct/.part-00000-f2547b28-9219-4628-8462-cc9c56edfebb-c000.snappy.parquet.crc b/presto-delta/src/test/resources/data-reader-nested-struct/.part-00000-f2547b28-9219-4628-8462-cc9c56edfebb-c000.snappy.parquet.crc new file mode 100644 index 000000000000..0dacf5190293 Binary files /dev/null and b/presto-delta/src/test/resources/data-reader-nested-struct/.part-00000-f2547b28-9219-4628-8462-cc9c56edfebb-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/data-reader-nested-struct/.part-00001-0f755735-3b5b-449a-8f93-92a40d9f065d-c000.snappy.parquet.crc b/presto-delta/src/test/resources/data-reader-nested-struct/.part-00001-0f755735-3b5b-449a-8f93-92a40d9f065d-c000.snappy.parquet.crc new file mode 100644 index 000000000000..976c62f866fe Binary files /dev/null and b/presto-delta/src/test/resources/data-reader-nested-struct/.part-00001-0f755735-3b5b-449a-8f93-92a40d9f065d-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/data-reader-nested-struct/_delta_log/00000000000000000000.json b/presto-delta/src/test/resources/data-reader-nested-struct/_delta_log/00000000000000000000.json new file mode 100644 index 000000000000..4046b145ff4f --- /dev/null +++ b/presto-delta/src/test/resources/data-reader-nested-struct/_delta_log/00000000000000000000.json @@ -0,0 +1,5 @@ +{"commitInfo":{"timestamp":1603724040818,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[]"},"isBlindAppend":true}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"975ef365-8dec-4bbf-ab88-264c10987001","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"a\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"aa\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"ab\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"ac\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"aca\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"acb\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"b\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1603724040747}} +{"add":{"path":"part-00000-f2547b28-9219-4628-8462-cc9c56edfebb-c000.snappy.parquet","partitionValues":{},"size":1432,"modificationTime":1603724040000,"dataChange":true}} +{"add":{"path":"part-00001-0f755735-3b5b-449a-8f93-92a40d9f065d-c000.snappy.parquet","partitionValues":{},"size":1439,"modificationTime":1603724040000,"dataChange":true}} diff --git a/presto-delta/src/test/resources/data-reader-nested-struct/part-00000-f2547b28-9219-4628-8462-cc9c56edfebb-c000.snappy.parquet b/presto-delta/src/test/resources/data-reader-nested-struct/part-00000-f2547b28-9219-4628-8462-cc9c56edfebb-c000.snappy.parquet new file mode 100644 index 000000000000..d1b86143ec41 Binary files /dev/null and b/presto-delta/src/test/resources/data-reader-nested-struct/part-00000-f2547b28-9219-4628-8462-cc9c56edfebb-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/data-reader-nested-struct/part-00001-0f755735-3b5b-449a-8f93-92a40d9f065d-c000.snappy.parquet b/presto-delta/src/test/resources/data-reader-nested-struct/part-00001-0f755735-3b5b-449a-8f93-92a40d9f065d-c000.snappy.parquet new file mode 100644 index 000000000000..b2114ea6d092 Binary files /dev/null and b/presto-delta/src/test/resources/data-reader-nested-struct/part-00001-0f755735-3b5b-449a-8f93-92a40d9f065d-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/data-reader-partition-values/_delta_log/00000000000000000000.crc b/presto-delta/src/test/resources/data-reader-partition-values/_delta_log/00000000000000000000.crc new file mode 100644 index 000000000000..db1f431a926e --- /dev/null +++ b/presto-delta/src/test/resources/data-reader-partition-values/_delta_log/00000000000000000000.crc @@ -0,0 +1 @@ +{"tableSizeBytes":1395,"numFiles":3,"numMetadata":1,"numProtocol":1,"numTransactions":0} diff --git a/presto-delta/src/test/resources/data-reader-partition-values/_delta_log/00000000000000000000.json b/presto-delta/src/test/resources/data-reader-partition-values/_delta_log/00000000000000000000.json new file mode 100644 index 000000000000..bb3fa5f66e96 --- /dev/null +++ b/presto-delta/src/test/resources/data-reader-partition-values/_delta_log/00000000000000000000.json @@ -0,0 +1,6 @@ +{"commitInfo":{"timestamp":1631515106977,"userId":"1619203104604657","userName":"wwang@talend.com","operation":"WRITE","operationParameters":{"mode":"Overwrite","partitionBy":"[\"as_int\",\"as_long\",\"as_byte\",\"as_short\",\"as_boolean\",\"as_float\",\"as_double\",\"as_string\",\"as_date\",\"as_timestamp\",\"as_big_decimal\"]"},"notebook":{"notebookId":"2452415333216186"},"clusterId":"0830-085441-merit621","isolationLevel":"WriteSerializable","isBlindAppend":false,"operationMetrics":{"numFiles":"3","numOutputBytes":"1395","numOutputRows":"3"}}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"71af55bf-be84-44f0-98dd-275f82b966d3","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"as_int\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"as_long\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"as_byte\",\"type\":\"byte\",\"nullable\":true,\"metadata\":{}},{\"name\":\"as_short\",\"type\":\"short\",\"nullable\":true,\"metadata\":{}},{\"name\":\"as_boolean\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"as_float\",\"type\":\"float\",\"nullable\":true,\"metadata\":{}},{\"name\":\"as_double\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"as_string\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"as_date\",\"type\":\"date\",\"nullable\":true,\"metadata\":{}},{\"name\":\"as_timestamp\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}},{\"name\":\"as_big_decimal\",\"type\":\"decimal(1,0)\",\"nullable\":true,\"metadata\":{}},{\"name\":\"value\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["as_int","as_long","as_byte","as_short","as_boolean","as_float","as_double","as_string","as_date","as_timestamp","as_big_decimal"],"configuration":{},"createdTime":1631515104797}} +{"add":{"path":"as_int=__HIVE_DEFAULT_PARTITION__/as_long=__HIVE_DEFAULT_PARTITION__/as_byte=__HIVE_DEFAULT_PARTITION__/as_short=__HIVE_DEFAULT_PARTITION__/as_boolean=__HIVE_DEFAULT_PARTITION__/as_float=__HIVE_DEFAULT_PARTITION__/as_double=__HIVE_DEFAULT_PARTITION__/as_string=__HIVE_DEFAULT_PARTITION__/as_date=__HIVE_DEFAULT_PARTITION__/as_timestamp=__HIVE_DEFAULT_PARTITION__/as_big_decimal=__HIVE_DEFAULT_PARTITION__/part-00002-b31c437d-93a7-462c-b88b-f8b49370d785.c000.snappy.parquet","partitionValues":{"as_big_decimal":null,"as_int":null,"as_byte":null,"as_long":null,"as_date":null,"as_string":null,"as_timestamp":null,"as_float":null,"as_short":null,"as_boolean":null,"as_double":null},"size":465,"modificationTime":1631515106000,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"value\":\"2\"},\"maxValues\":{\"value\":\"2\"},\"nullCount\":{\"value\":0}}"}} +{"add":{"path":"as_int=0/as_long=0/as_byte=0/as_short=0/as_boolean=true/as_float=0.0/as_double=0.0/as_string=0/as_date=2021-09-08/as_timestamp=2021-09-08%2011%253A11%253A11/as_big_decimal=0/part-00005-2d4e572d-5bdd-43f4-9d13-7c5354deb1f6.c000.snappy.parquet","partitionValues":{"as_big_decimal":"0","as_int":"0","as_byte":"0","as_long":"0","as_date":"2021-09-08","as_string":"0","as_timestamp":"2021-09-08 11:11:11","as_float":"0.0","as_short":"0","as_boolean":"true","as_double":"0.0"},"size":465,"modificationTime":1631515106000,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"value\":\"0\"},\"maxValues\":{\"value\":\"0\"},\"nullCount\":{\"value\":0}}"}} +{"add":{"path":"as_int=1/as_long=1/as_byte=1/as_short=1/as_boolean=false/as_float=1.0/as_double=1.0/as_string=1/as_date=2021-09-08/as_timestamp=2021-09-08%2011%253A11%253A11/as_big_decimal=1/part-00007-00fd7022-5e9c-4cba-b8c8-2297ef36e5ff.c000.snappy.parquet","partitionValues":{"as_big_decimal":"1","as_int":"1","as_byte":"1","as_long":"1","as_date":"2021-09-08","as_string":"1","as_timestamp":"2021-09-08 11:11:11","as_float":"1.0","as_short":"1","as_boolean":"false","as_double":"1.0"},"size":465,"modificationTime":1631515106000,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"value\":\"1\"},\"maxValues\":{\"value\":\"1\"},\"nullCount\":{\"value\":0}}"}} diff --git a/presto-delta/src/test/resources/data-reader-partition-values/as_int=0/as_long=0/as_byte=0/as_short=0/as_boolean=true/as_float=0.0/as_double=0.0/as_string=0/as_date=2021-09-08/as_timestamp=2021-09-08 11%3A11%3A11/as_big_decimal=0/part-00005-2d4e572d-5bdd-43f4-9d13-7c5354deb1f6.c000.snappy.parquet b/presto-delta/src/test/resources/data-reader-partition-values/as_int=0/as_long=0/as_byte=0/as_short=0/as_boolean=true/as_float=0.0/as_double=0.0/as_string=0/as_date=2021-09-08/as_timestamp=2021-09-08 11%3A11%3A11/as_big_decimal=0/part-00005-2d4e572d-5bdd-43f4-9d13-7c5354deb1f6.c000.snappy.parquet new file mode 100644 index 000000000000..0ff0067f430f Binary files /dev/null and b/presto-delta/src/test/resources/data-reader-partition-values/as_int=0/as_long=0/as_byte=0/as_short=0/as_boolean=true/as_float=0.0/as_double=0.0/as_string=0/as_date=2021-09-08/as_timestamp=2021-09-08 11%3A11%3A11/as_big_decimal=0/part-00005-2d4e572d-5bdd-43f4-9d13-7c5354deb1f6.c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/data-reader-partition-values/as_int=1/as_long=1/as_byte=1/as_short=1/as_boolean=false/as_float=1.0/as_double=1.0/as_string=1/as_date=2021-09-08/as_timestamp=2021-09-08 11%3A11%3A11/as_big_decimal=1/part-00007-00fd7022-5e9c-4cba-b8c8-2297ef36e5ff.c000.snappy.parquet b/presto-delta/src/test/resources/data-reader-partition-values/as_int=1/as_long=1/as_byte=1/as_short=1/as_boolean=false/as_float=1.0/as_double=1.0/as_string=1/as_date=2021-09-08/as_timestamp=2021-09-08 11%3A11%3A11/as_big_decimal=1/part-00007-00fd7022-5e9c-4cba-b8c8-2297ef36e5ff.c000.snappy.parquet new file mode 100644 index 000000000000..dee0f7863329 Binary files /dev/null and b/presto-delta/src/test/resources/data-reader-partition-values/as_int=1/as_long=1/as_byte=1/as_short=1/as_boolean=false/as_float=1.0/as_double=1.0/as_string=1/as_date=2021-09-08/as_timestamp=2021-09-08 11%3A11%3A11/as_big_decimal=1/part-00007-00fd7022-5e9c-4cba-b8c8-2297ef36e5ff.c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/data-reader-partition-values/as_int=__HIVE_DEFAULT_PARTITION__/as_long=__HIVE_DEFAULT_PARTITION__/as_byte=__HIVE_DEFAULT_PARTITION__/as_short=__HIVE_DEFAULT_PARTITION__/as_boolean=__HIVE_DEFAULT_PARTITION__/as_float=__HIVE_DEFAULT_PARTITION__/as_double=__HIVE_DEFAULT_PARTITION__/as_string=__HIVE_DEFAULT_PARTITION__/as_date=__HIVE_DEFAULT_PARTITION__/as_timestamp=__HIVE_DEFAULT_PARTITION__/as_big_decimal=__HIVE_DEFAULT_PARTITION__/part-00002-b31c437d-93a7-462c-b88b-f8b49370d785.c000.snappy.parquet b/presto-delta/src/test/resources/data-reader-partition-values/as_int=__HIVE_DEFAULT_PARTITION__/as_long=__HIVE_DEFAULT_PARTITION__/as_byte=__HIVE_DEFAULT_PARTITION__/as_short=__HIVE_DEFAULT_PARTITION__/as_boolean=__HIVE_DEFAULT_PARTITION__/as_float=__HIVE_DEFAULT_PARTITION__/as_double=__HIVE_DEFAULT_PARTITION__/as_string=__HIVE_DEFAULT_PARTITION__/as_date=__HIVE_DEFAULT_PARTITION__/as_timestamp=__HIVE_DEFAULT_PARTITION__/as_big_decimal=__HIVE_DEFAULT_PARTITION__/part-00002-b31c437d-93a7-462c-b88b-f8b49370d785.c000.snappy.parquet new file mode 100644 index 000000000000..07cce42752a2 Binary files /dev/null and b/presto-delta/src/test/resources/data-reader-partition-values/as_int=__HIVE_DEFAULT_PARTITION__/as_long=__HIVE_DEFAULT_PARTITION__/as_byte=__HIVE_DEFAULT_PARTITION__/as_short=__HIVE_DEFAULT_PARTITION__/as_boolean=__HIVE_DEFAULT_PARTITION__/as_float=__HIVE_DEFAULT_PARTITION__/as_double=__HIVE_DEFAULT_PARTITION__/as_string=__HIVE_DEFAULT_PARTITION__/as_date=__HIVE_DEFAULT_PARTITION__/as_timestamp=__HIVE_DEFAULT_PARTITION__/as_big_decimal=__HIVE_DEFAULT_PARTITION__/part-00002-b31c437d-93a7-462c-b88b-f8b49370d785.c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/data-reader-primitives/.part-00000-4f2f0b9f-50b3-4e7b-96a1-e2bb0f246b06-c000.snappy.parquet.crc b/presto-delta/src/test/resources/data-reader-primitives/.part-00000-4f2f0b9f-50b3-4e7b-96a1-e2bb0f246b06-c000.snappy.parquet.crc new file mode 100644 index 000000000000..11f8928c2751 Binary files /dev/null and b/presto-delta/src/test/resources/data-reader-primitives/.part-00000-4f2f0b9f-50b3-4e7b-96a1-e2bb0f246b06-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/data-reader-primitives/.part-00001-09e47b80-36c2-4475-a810-fbd8e7994971-c000.snappy.parquet.crc b/presto-delta/src/test/resources/data-reader-primitives/.part-00001-09e47b80-36c2-4475-a810-fbd8e7994971-c000.snappy.parquet.crc new file mode 100644 index 000000000000..852ffc4e2fc8 Binary files /dev/null and b/presto-delta/src/test/resources/data-reader-primitives/.part-00001-09e47b80-36c2-4475-a810-fbd8e7994971-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/data-reader-primitives/_delta_log/00000000000000000000.json b/presto-delta/src/test/resources/data-reader-primitives/_delta_log/00000000000000000000.json new file mode 100644 index 000000000000..9c9a0d11558f --- /dev/null +++ b/presto-delta/src/test/resources/data-reader-primitives/_delta_log/00000000000000000000.json @@ -0,0 +1,5 @@ +{"commitInfo":{"timestamp":1607520163636,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[]"},"isBlindAppend":true,"operationMetrics":{"numFiles":"2","numOutputBytes":"5050","numOutputRows":"11"}}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"testId","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"as_int\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"as_long\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"as_byte\",\"type\":\"byte\",\"nullable\":true,\"metadata\":{}},{\"name\":\"as_short\",\"type\":\"short\",\"nullable\":true,\"metadata\":{}},{\"name\":\"as_boolean\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"as_float\",\"type\":\"float\",\"nullable\":true,\"metadata\":{}},{\"name\":\"as_double\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"as_string\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"as_binary\",\"type\":\"binary\",\"nullable\":true,\"metadata\":{}},{\"name\":\"as_big_decimal\",\"type\":\"decimal(1,0)\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1607520161353}} +{"add":{"path":"part-00000-4f2f0b9f-50b3-4e7b-96a1-e2bb0f246b06-c000.snappy.parquet","partitionValues":{},"size":2482,"modificationTime":1607520163000,"dataChange":true}} +{"add":{"path":"part-00001-09e47b80-36c2-4475-a810-fbd8e7994971-c000.snappy.parquet","partitionValues":{},"size":2568,"modificationTime":1607520163000,"dataChange":true}} diff --git a/presto-delta/src/test/resources/data-reader-primitives/part-00000-4f2f0b9f-50b3-4e7b-96a1-e2bb0f246b06-c000.snappy.parquet b/presto-delta/src/test/resources/data-reader-primitives/part-00000-4f2f0b9f-50b3-4e7b-96a1-e2bb0f246b06-c000.snappy.parquet new file mode 100644 index 000000000000..b0442b0085a0 Binary files /dev/null and b/presto-delta/src/test/resources/data-reader-primitives/part-00000-4f2f0b9f-50b3-4e7b-96a1-e2bb0f246b06-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/data-reader-primitives/part-00001-09e47b80-36c2-4475-a810-fbd8e7994971-c000.snappy.parquet b/presto-delta/src/test/resources/data-reader-primitives/part-00001-09e47b80-36c2-4475-a810-fbd8e7994971-c000.snappy.parquet new file mode 100644 index 000000000000..745394ca6c09 Binary files /dev/null and b/presto-delta/src/test/resources/data-reader-primitives/part-00001-09e47b80-36c2-4475-a810-fbd8e7994971-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/deltatbl-partition-prune/_delta_log/00000000000000000000.json b/presto-delta/src/test/resources/deltatbl-partition-prune/_delta_log/00000000000000000000.json new file mode 100644 index 000000000000..88b46f313a20 --- /dev/null +++ b/presto-delta/src/test/resources/deltatbl-partition-prune/_delta_log/00000000000000000000.json @@ -0,0 +1,8 @@ +{"commitInfo":{"timestamp":1629873077420,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[\"date\",\"city\"]"},"isBlindAppend":true,"operationMetrics":{"numFiles":"5","numOutputBytes":"3195","numOutputRows":"5"}}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"testId","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"city\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"date\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"cnt\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["date","city"],"configuration":{},"createdTime":1629873075437}} +{"add":{"path":"date=20180520/city=hz/part-00000-de1d5bcd-ad7e-4b88-ba9b-31fb8aeb8093.c000.snappy.parquet","partitionValues":{"date":"20180520","city":"hz"},"size":628,"modificationTime":1629873077000,"dataChange":true}} +{"add":{"path":"date=20180718/city=hz/part-00000-f888e95b-c831-43fe-bba8-3dbf43b4eb86.c000.snappy.parquet","partitionValues":{"date":"20180718","city":"hz"},"size":639,"modificationTime":1629873077000,"dataChange":true}} +{"add":{"path":"date=20180512/city=sh/part-00001-c87aeb63-6d9c-4511-b8b3-71d02178554f.c000.snappy.parquet","partitionValues":{"date":"20180512","city":"sh"},"size":628,"modificationTime":1629873077000,"dataChange":true}} +{"add":{"path":"date=20180520/city=bj/part-00001-4c732f0f-a473-400a-8ba3-1499f599b8f1.c000.snappy.parquet","partitionValues":{"date":"20180520","city":"bj"},"size":650,"modificationTime":1629873077000,"dataChange":true}} +{"add":{"path":"date=20181212/city=sz/part-00001-529ff89b-55c6-4405-a6cc-04759d5f692b.c000.snappy.parquet","partitionValues":{"date":"20181212","city":"sz"},"size":650,"modificationTime":1629873077000,"dataChange":true}} diff --git a/presto-delta/src/test/resources/deltatbl-partition-prune/date=20180512/city=sh/.part-00001-c87aeb63-6d9c-4511-b8b3-71d02178554f.c000.snappy.parquet.crc b/presto-delta/src/test/resources/deltatbl-partition-prune/date=20180512/city=sh/.part-00001-c87aeb63-6d9c-4511-b8b3-71d02178554f.c000.snappy.parquet.crc new file mode 100644 index 000000000000..e301f348551c Binary files /dev/null and b/presto-delta/src/test/resources/deltatbl-partition-prune/date=20180512/city=sh/.part-00001-c87aeb63-6d9c-4511-b8b3-71d02178554f.c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/deltatbl-partition-prune/date=20180512/city=sh/part-00001-c87aeb63-6d9c-4511-b8b3-71d02178554f.c000.snappy.parquet b/presto-delta/src/test/resources/deltatbl-partition-prune/date=20180512/city=sh/part-00001-c87aeb63-6d9c-4511-b8b3-71d02178554f.c000.snappy.parquet new file mode 100644 index 000000000000..5a2666d2f138 Binary files /dev/null and b/presto-delta/src/test/resources/deltatbl-partition-prune/date=20180512/city=sh/part-00001-c87aeb63-6d9c-4511-b8b3-71d02178554f.c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/deltatbl-partition-prune/date=20180520/city=bj/.part-00001-4c732f0f-a473-400a-8ba3-1499f599b8f1.c000.snappy.parquet.crc b/presto-delta/src/test/resources/deltatbl-partition-prune/date=20180520/city=bj/.part-00001-4c732f0f-a473-400a-8ba3-1499f599b8f1.c000.snappy.parquet.crc new file mode 100644 index 000000000000..8c26ac11ed50 Binary files /dev/null and b/presto-delta/src/test/resources/deltatbl-partition-prune/date=20180520/city=bj/.part-00001-4c732f0f-a473-400a-8ba3-1499f599b8f1.c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/deltatbl-partition-prune/date=20180520/city=bj/part-00001-4c732f0f-a473-400a-8ba3-1499f599b8f1.c000.snappy.parquet b/presto-delta/src/test/resources/deltatbl-partition-prune/date=20180520/city=bj/part-00001-4c732f0f-a473-400a-8ba3-1499f599b8f1.c000.snappy.parquet new file mode 100644 index 000000000000..4787f1ca94e6 Binary files /dev/null and b/presto-delta/src/test/resources/deltatbl-partition-prune/date=20180520/city=bj/part-00001-4c732f0f-a473-400a-8ba3-1499f599b8f1.c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/deltatbl-partition-prune/date=20180520/city=hz/.part-00000-de1d5bcd-ad7e-4b88-ba9b-31fb8aeb8093.c000.snappy.parquet.crc b/presto-delta/src/test/resources/deltatbl-partition-prune/date=20180520/city=hz/.part-00000-de1d5bcd-ad7e-4b88-ba9b-31fb8aeb8093.c000.snappy.parquet.crc new file mode 100644 index 000000000000..ed9072bf2dd1 Binary files /dev/null and b/presto-delta/src/test/resources/deltatbl-partition-prune/date=20180520/city=hz/.part-00000-de1d5bcd-ad7e-4b88-ba9b-31fb8aeb8093.c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/deltatbl-partition-prune/date=20180520/city=hz/part-00000-de1d5bcd-ad7e-4b88-ba9b-31fb8aeb8093.c000.snappy.parquet b/presto-delta/src/test/resources/deltatbl-partition-prune/date=20180520/city=hz/part-00000-de1d5bcd-ad7e-4b88-ba9b-31fb8aeb8093.c000.snappy.parquet new file mode 100644 index 000000000000..585f1f0a6d1e Binary files /dev/null and b/presto-delta/src/test/resources/deltatbl-partition-prune/date=20180520/city=hz/part-00000-de1d5bcd-ad7e-4b88-ba9b-31fb8aeb8093.c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/deltatbl-partition-prune/date=20180718/city=hz/.part-00000-f888e95b-c831-43fe-bba8-3dbf43b4eb86.c000.snappy.parquet.crc b/presto-delta/src/test/resources/deltatbl-partition-prune/date=20180718/city=hz/.part-00000-f888e95b-c831-43fe-bba8-3dbf43b4eb86.c000.snappy.parquet.crc new file mode 100644 index 000000000000..811ff4f5a12e Binary files /dev/null and b/presto-delta/src/test/resources/deltatbl-partition-prune/date=20180718/city=hz/.part-00000-f888e95b-c831-43fe-bba8-3dbf43b4eb86.c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/deltatbl-partition-prune/date=20180718/city=hz/part-00000-f888e95b-c831-43fe-bba8-3dbf43b4eb86.c000.snappy.parquet b/presto-delta/src/test/resources/deltatbl-partition-prune/date=20180718/city=hz/part-00000-f888e95b-c831-43fe-bba8-3dbf43b4eb86.c000.snappy.parquet new file mode 100644 index 000000000000..019d84d3fc80 Binary files /dev/null and b/presto-delta/src/test/resources/deltatbl-partition-prune/date=20180718/city=hz/part-00000-f888e95b-c831-43fe-bba8-3dbf43b4eb86.c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/deltatbl-partition-prune/date=20181212/city=sz/.part-00001-529ff89b-55c6-4405-a6cc-04759d5f692b.c000.snappy.parquet.crc b/presto-delta/src/test/resources/deltatbl-partition-prune/date=20181212/city=sz/.part-00001-529ff89b-55c6-4405-a6cc-04759d5f692b.c000.snappy.parquet.crc new file mode 100644 index 000000000000..8c754a4761e9 Binary files /dev/null and b/presto-delta/src/test/resources/deltatbl-partition-prune/date=20181212/city=sz/.part-00001-529ff89b-55c6-4405-a6cc-04759d5f692b.c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/deltatbl-partition-prune/date=20181212/city=sz/part-00001-529ff89b-55c6-4405-a6cc-04759d5f692b.c000.snappy.parquet b/presto-delta/src/test/resources/deltatbl-partition-prune/date=20181212/city=sz/part-00001-529ff89b-55c6-4405-a6cc-04759d5f692b.c000.snappy.parquet new file mode 100644 index 000000000000..b35457e5e39b Binary files /dev/null and b/presto-delta/src/test/resources/deltatbl-partition-prune/date=20181212/city=sz/part-00001-529ff89b-55c6-4405-a6cc-04759d5f692b.c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/deltatbl-partitioned/_delta_log/00000000000000000000.json b/presto-delta/src/test/resources/deltatbl-partitioned/_delta_log/00000000000000000000.json new file mode 100644 index 000000000000..df728b31bb46 --- /dev/null +++ b/presto-delta/src/test/resources/deltatbl-partitioned/_delta_log/00000000000000000000.json @@ -0,0 +1,7 @@ +{"commitInfo":{"timestamp":1629873032991,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[\"c2\"]"},"isBlindAppend":true,"operationMetrics":{"numFiles":"4","numOutputBytes":"1734","numOutputRows":"10"}}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"testId","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"c1\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c2\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["c2"],"configuration":{},"createdTime":1629873029858}} +{"add":{"path":"c2=foo0/part-00000-2bcc9ff6-0551-4401-bd22-d361a60627e3.c000.snappy.parquet","partitionValues":{"c2":"foo0"},"size":436,"modificationTime":1629873032000,"dataChange":true}} +{"add":{"path":"c2=foo1/part-00000-786c7455-9587-454f-9a4c-de0b22b62bbd.c000.snappy.parquet","partitionValues":{"c2":"foo1"},"size":431,"modificationTime":1629873032000,"dataChange":true}} +{"add":{"path":"c2=foo0/part-00001-ca647ee7-f1ad-4d70-bf02-5d1872324d6f.c000.snappy.parquet","partitionValues":{"c2":"foo0"},"size":431,"modificationTime":1629873032000,"dataChange":true}} +{"add":{"path":"c2=foo1/part-00001-1c702e73-89b5-465a-9c6a-25f7559cd150.c000.snappy.parquet","partitionValues":{"c2":"foo1"},"size":436,"modificationTime":1629873032000,"dataChange":true}} diff --git a/presto-delta/src/test/resources/deltatbl-partitioned/c2=foo0/.part-00000-2bcc9ff6-0551-4401-bd22-d361a60627e3.c000.snappy.parquet.crc b/presto-delta/src/test/resources/deltatbl-partitioned/c2=foo0/.part-00000-2bcc9ff6-0551-4401-bd22-d361a60627e3.c000.snappy.parquet.crc new file mode 100644 index 000000000000..36aeb8d3d89c Binary files /dev/null and b/presto-delta/src/test/resources/deltatbl-partitioned/c2=foo0/.part-00000-2bcc9ff6-0551-4401-bd22-d361a60627e3.c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/deltatbl-partitioned/c2=foo0/.part-00001-ca647ee7-f1ad-4d70-bf02-5d1872324d6f.c000.snappy.parquet.crc b/presto-delta/src/test/resources/deltatbl-partitioned/c2=foo0/.part-00001-ca647ee7-f1ad-4d70-bf02-5d1872324d6f.c000.snappy.parquet.crc new file mode 100644 index 000000000000..dc43e92d33a4 Binary files /dev/null and b/presto-delta/src/test/resources/deltatbl-partitioned/c2=foo0/.part-00001-ca647ee7-f1ad-4d70-bf02-5d1872324d6f.c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/deltatbl-partitioned/c2=foo0/part-00000-2bcc9ff6-0551-4401-bd22-d361a60627e3.c000.snappy.parquet b/presto-delta/src/test/resources/deltatbl-partitioned/c2=foo0/part-00000-2bcc9ff6-0551-4401-bd22-d361a60627e3.c000.snappy.parquet new file mode 100644 index 000000000000..c7bda6510def Binary files /dev/null and b/presto-delta/src/test/resources/deltatbl-partitioned/c2=foo0/part-00000-2bcc9ff6-0551-4401-bd22-d361a60627e3.c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/deltatbl-partitioned/c2=foo0/part-00001-ca647ee7-f1ad-4d70-bf02-5d1872324d6f.c000.snappy.parquet b/presto-delta/src/test/resources/deltatbl-partitioned/c2=foo0/part-00001-ca647ee7-f1ad-4d70-bf02-5d1872324d6f.c000.snappy.parquet new file mode 100644 index 000000000000..2297174763f7 Binary files /dev/null and b/presto-delta/src/test/resources/deltatbl-partitioned/c2=foo0/part-00001-ca647ee7-f1ad-4d70-bf02-5d1872324d6f.c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/deltatbl-partitioned/c2=foo1/.part-00000-786c7455-9587-454f-9a4c-de0b22b62bbd.c000.snappy.parquet.crc b/presto-delta/src/test/resources/deltatbl-partitioned/c2=foo1/.part-00000-786c7455-9587-454f-9a4c-de0b22b62bbd.c000.snappy.parquet.crc new file mode 100644 index 000000000000..079d5bd1a8c8 Binary files /dev/null and b/presto-delta/src/test/resources/deltatbl-partitioned/c2=foo1/.part-00000-786c7455-9587-454f-9a4c-de0b22b62bbd.c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/deltatbl-partitioned/c2=foo1/.part-00001-1c702e73-89b5-465a-9c6a-25f7559cd150.c000.snappy.parquet.crc b/presto-delta/src/test/resources/deltatbl-partitioned/c2=foo1/.part-00001-1c702e73-89b5-465a-9c6a-25f7559cd150.c000.snappy.parquet.crc new file mode 100644 index 000000000000..af3ce1bf8a8c Binary files /dev/null and b/presto-delta/src/test/resources/deltatbl-partitioned/c2=foo1/.part-00001-1c702e73-89b5-465a-9c6a-25f7559cd150.c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/deltatbl-partitioned/c2=foo1/part-00000-786c7455-9587-454f-9a4c-de0b22b62bbd.c000.snappy.parquet b/presto-delta/src/test/resources/deltatbl-partitioned/c2=foo1/part-00000-786c7455-9587-454f-9a4c-de0b22b62bbd.c000.snappy.parquet new file mode 100644 index 000000000000..78c4d0f46d0b Binary files /dev/null and b/presto-delta/src/test/resources/deltatbl-partitioned/c2=foo1/part-00000-786c7455-9587-454f-9a4c-de0b22b62bbd.c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/deltatbl-partitioned/c2=foo1/part-00001-1c702e73-89b5-465a-9c6a-25f7559cd150.c000.snappy.parquet b/presto-delta/src/test/resources/deltatbl-partitioned/c2=foo1/part-00001-1c702e73-89b5-465a-9c6a-25f7559cd150.c000.snappy.parquet new file mode 100644 index 000000000000..8143adb67f74 Binary files /dev/null and b/presto-delta/src/test/resources/deltatbl-partitioned/c2=foo1/part-00001-1c702e73-89b5-465a-9c6a-25f7559cd150.c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/snapshot-data3/.part-00000-0441e99a-c421-400e-83a1-212aa6c84c73-c000.snappy.parquet.crc b/presto-delta/src/test/resources/snapshot-data3/.part-00000-0441e99a-c421-400e-83a1-212aa6c84c73-c000.snappy.parquet.crc new file mode 100644 index 000000000000..f66b5b65e0e6 Binary files /dev/null and b/presto-delta/src/test/resources/snapshot-data3/.part-00000-0441e99a-c421-400e-83a1-212aa6c84c73-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/snapshot-data3/.part-00000-64680d94-9e18-4fa1-9ca9-f0cd8a9cfd11-c000.snappy.parquet.crc b/presto-delta/src/test/resources/snapshot-data3/.part-00000-64680d94-9e18-4fa1-9ca9-f0cd8a9cfd11-c000.snappy.parquet.crc new file mode 100644 index 000000000000..02716266145c Binary files /dev/null and b/presto-delta/src/test/resources/snapshot-data3/.part-00000-64680d94-9e18-4fa1-9ca9-f0cd8a9cfd11-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/snapshot-data3/.part-00000-842017c2-3e02-44b5-a3d6-5b9ae1745045-c000.snappy.parquet.crc b/presto-delta/src/test/resources/snapshot-data3/.part-00000-842017c2-3e02-44b5-a3d6-5b9ae1745045-c000.snappy.parquet.crc new file mode 100644 index 000000000000..4f8c36bdd094 Binary files /dev/null and b/presto-delta/src/test/resources/snapshot-data3/.part-00000-842017c2-3e02-44b5-a3d6-5b9ae1745045-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/snapshot-data3/.part-00000-cb078bc1-0aeb-46ed-9cf8-74a843b32c8c-c000.snappy.parquet.crc b/presto-delta/src/test/resources/snapshot-data3/.part-00000-cb078bc1-0aeb-46ed-9cf8-74a843b32c8c-c000.snappy.parquet.crc new file mode 100644 index 000000000000..2b1ab4061b33 Binary files /dev/null and b/presto-delta/src/test/resources/snapshot-data3/.part-00000-cb078bc1-0aeb-46ed-9cf8-74a843b32c8c-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/snapshot-data3/.part-00001-34c8c673-3f44-4fa7-b94e-07357ec28a7d-c000.snappy.parquet.crc b/presto-delta/src/test/resources/snapshot-data3/.part-00001-34c8c673-3f44-4fa7-b94e-07357ec28a7d-c000.snappy.parquet.crc new file mode 100644 index 000000000000..0ea3aa2df28c Binary files /dev/null and b/presto-delta/src/test/resources/snapshot-data3/.part-00001-34c8c673-3f44-4fa7-b94e-07357ec28a7d-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/snapshot-data3/.part-00001-9bf4b8f8-1b95-411b-bf10-28dc03aa9d2f-c000.snappy.parquet.crc b/presto-delta/src/test/resources/snapshot-data3/.part-00001-9bf4b8f8-1b95-411b-bf10-28dc03aa9d2f-c000.snappy.parquet.crc new file mode 100644 index 000000000000..692cee1e438a Binary files /dev/null and b/presto-delta/src/test/resources/snapshot-data3/.part-00001-9bf4b8f8-1b95-411b-bf10-28dc03aa9d2f-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/snapshot-data3/.part-00001-b8249b87-0b7a-4461-8a8a-fa958802b523-c000.snappy.parquet.crc b/presto-delta/src/test/resources/snapshot-data3/.part-00001-b8249b87-0b7a-4461-8a8a-fa958802b523-c000.snappy.parquet.crc new file mode 100644 index 000000000000..7db783ac002c Binary files /dev/null and b/presto-delta/src/test/resources/snapshot-data3/.part-00001-b8249b87-0b7a-4461-8a8a-fa958802b523-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/snapshot-data3/.part-00001-e62ca5a1-923c-4ee6-998b-c61d1cfb0b1c-c000.snappy.parquet.crc b/presto-delta/src/test/resources/snapshot-data3/.part-00001-e62ca5a1-923c-4ee6-998b-c61d1cfb0b1c-c000.snappy.parquet.crc new file mode 100644 index 000000000000..5c04eeeb2ec1 Binary files /dev/null and b/presto-delta/src/test/resources/snapshot-data3/.part-00001-e62ca5a1-923c-4ee6-998b-c61d1cfb0b1c-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/snapshot-data3/_delta_log/00000000000000000000.json b/presto-delta/src/test/resources/snapshot-data3/_delta_log/00000000000000000000.json new file mode 100644 index 000000000000..491571a22c3f --- /dev/null +++ b/presto-delta/src/test/resources/snapshot-data3/_delta_log/00000000000000000000.json @@ -0,0 +1,5 @@ +{"commitInfo":{"timestamp":1603723967632,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"isBlindAppend":true}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"93351cf1-c931-4326-88f0-d10e29e71b21","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"col1\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"col2\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1603723967515}} +{"add":{"path":"part-00000-0441e99a-c421-400e-83a1-212aa6c84c73-c000.snappy.parquet","partitionValues":{},"size":650,"modificationTime":1603723967000,"dataChange":true}} +{"add":{"path":"part-00001-34c8c673-3f44-4fa7-b94e-07357ec28a7d-c000.snappy.parquet","partitionValues":{},"size":650,"modificationTime":1603723967000,"dataChange":true}} diff --git a/presto-delta/src/test/resources/snapshot-data3/_delta_log/00000000000000000001.json b/presto-delta/src/test/resources/snapshot-data3/_delta_log/00000000000000000001.json new file mode 100644 index 000000000000..db30a2fad634 --- /dev/null +++ b/presto-delta/src/test/resources/snapshot-data3/_delta_log/00000000000000000001.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1603723969055,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":0,"isBlindAppend":true}} +{"add":{"path":"part-00000-64680d94-9e18-4fa1-9ca9-f0cd8a9cfd11-c000.snappy.parquet","partitionValues":{},"size":649,"modificationTime":1603723969000,"dataChange":true}} +{"add":{"path":"part-00001-b8249b87-0b7a-4461-8a8a-fa958802b523-c000.snappy.parquet","partitionValues":{},"size":649,"modificationTime":1603723969000,"dataChange":true}} diff --git a/presto-delta/src/test/resources/snapshot-data3/_delta_log/00000000000000000002.json b/presto-delta/src/test/resources/snapshot-data3/_delta_log/00000000000000000002.json new file mode 100644 index 000000000000..53eb414f095b --- /dev/null +++ b/presto-delta/src/test/resources/snapshot-data3/_delta_log/00000000000000000002.json @@ -0,0 +1,7 @@ +{"commitInfo":{"timestamp":1603723970832,"operation":"WRITE","operationParameters":{"mode":"Overwrite","partitionBy":"[]"},"readVersion":1,"isBlindAppend":false}} +{"add":{"path":"part-00000-842017c2-3e02-44b5-a3d6-5b9ae1745045-c000.snappy.parquet","partitionValues":{},"size":649,"modificationTime":1603723970000,"dataChange":true}} +{"add":{"path":"part-00001-e62ca5a1-923c-4ee6-998b-c61d1cfb0b1c-c000.snappy.parquet","partitionValues":{},"size":649,"modificationTime":1603723970000,"dataChange":true}} +{"remove":{"path":"part-00000-0441e99a-c421-400e-83a1-212aa6c84c73-c000.snappy.parquet","deletionTimestamp":1603723970832,"dataChange":true}} +{"remove":{"path":"part-00000-64680d94-9e18-4fa1-9ca9-f0cd8a9cfd11-c000.snappy.parquet","deletionTimestamp":1603723970832,"dataChange":true}} +{"remove":{"path":"part-00001-b8249b87-0b7a-4461-8a8a-fa958802b523-c000.snappy.parquet","deletionTimestamp":1603723970832,"dataChange":true}} +{"remove":{"path":"part-00001-34c8c673-3f44-4fa7-b94e-07357ec28a7d-c000.snappy.parquet","deletionTimestamp":1603723970832,"dataChange":true}} diff --git a/presto-delta/src/test/resources/snapshot-data3/_delta_log/00000000000000000003.json b/presto-delta/src/test/resources/snapshot-data3/_delta_log/00000000000000000003.json new file mode 100644 index 000000000000..6ba370acbe8c --- /dev/null +++ b/presto-delta/src/test/resources/snapshot-data3/_delta_log/00000000000000000003.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1603723972251,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":2,"isBlindAppend":true}} +{"add":{"path":"part-00000-cb078bc1-0aeb-46ed-9cf8-74a843b32c8c-c000.snappy.parquet","partitionValues":{},"size":687,"modificationTime":1603723972000,"dataChange":true}} +{"add":{"path":"part-00001-9bf4b8f8-1b95-411b-bf10-28dc03aa9d2f-c000.snappy.parquet","partitionValues":{},"size":705,"modificationTime":1603723972000,"dataChange":true}} diff --git a/presto-delta/src/test/resources/snapshot-data3/part-00000-0441e99a-c421-400e-83a1-212aa6c84c73-c000.snappy.parquet b/presto-delta/src/test/resources/snapshot-data3/part-00000-0441e99a-c421-400e-83a1-212aa6c84c73-c000.snappy.parquet new file mode 100644 index 000000000000..7d1ff564e1e6 Binary files /dev/null and b/presto-delta/src/test/resources/snapshot-data3/part-00000-0441e99a-c421-400e-83a1-212aa6c84c73-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/snapshot-data3/part-00000-64680d94-9e18-4fa1-9ca9-f0cd8a9cfd11-c000.snappy.parquet b/presto-delta/src/test/resources/snapshot-data3/part-00000-64680d94-9e18-4fa1-9ca9-f0cd8a9cfd11-c000.snappy.parquet new file mode 100644 index 000000000000..198eea06e5f1 Binary files /dev/null and b/presto-delta/src/test/resources/snapshot-data3/part-00000-64680d94-9e18-4fa1-9ca9-f0cd8a9cfd11-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/snapshot-data3/part-00000-842017c2-3e02-44b5-a3d6-5b9ae1745045-c000.snappy.parquet b/presto-delta/src/test/resources/snapshot-data3/part-00000-842017c2-3e02-44b5-a3d6-5b9ae1745045-c000.snappy.parquet new file mode 100644 index 000000000000..81cc32d32618 Binary files /dev/null and b/presto-delta/src/test/resources/snapshot-data3/part-00000-842017c2-3e02-44b5-a3d6-5b9ae1745045-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/snapshot-data3/part-00000-cb078bc1-0aeb-46ed-9cf8-74a843b32c8c-c000.snappy.parquet b/presto-delta/src/test/resources/snapshot-data3/part-00000-cb078bc1-0aeb-46ed-9cf8-74a843b32c8c-c000.snappy.parquet new file mode 100644 index 000000000000..24104a72ca74 Binary files /dev/null and b/presto-delta/src/test/resources/snapshot-data3/part-00000-cb078bc1-0aeb-46ed-9cf8-74a843b32c8c-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/snapshot-data3/part-00001-34c8c673-3f44-4fa7-b94e-07357ec28a7d-c000.snappy.parquet b/presto-delta/src/test/resources/snapshot-data3/part-00001-34c8c673-3f44-4fa7-b94e-07357ec28a7d-c000.snappy.parquet new file mode 100644 index 000000000000..43b74e3d07f8 Binary files /dev/null and b/presto-delta/src/test/resources/snapshot-data3/part-00001-34c8c673-3f44-4fa7-b94e-07357ec28a7d-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/snapshot-data3/part-00001-9bf4b8f8-1b95-411b-bf10-28dc03aa9d2f-c000.snappy.parquet b/presto-delta/src/test/resources/snapshot-data3/part-00001-9bf4b8f8-1b95-411b-bf10-28dc03aa9d2f-c000.snappy.parquet new file mode 100644 index 000000000000..33d14ac58275 Binary files /dev/null and b/presto-delta/src/test/resources/snapshot-data3/part-00001-9bf4b8f8-1b95-411b-bf10-28dc03aa9d2f-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/snapshot-data3/part-00001-b8249b87-0b7a-4461-8a8a-fa958802b523-c000.snappy.parquet b/presto-delta/src/test/resources/snapshot-data3/part-00001-b8249b87-0b7a-4461-8a8a-fa958802b523-c000.snappy.parquet new file mode 100644 index 000000000000..6f0c663ee61a Binary files /dev/null and b/presto-delta/src/test/resources/snapshot-data3/part-00001-b8249b87-0b7a-4461-8a8a-fa958802b523-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/snapshot-data3/part-00001-e62ca5a1-923c-4ee6-998b-c61d1cfb0b1c-c000.snappy.parquet b/presto-delta/src/test/resources/snapshot-data3/part-00001-e62ca5a1-923c-4ee6-998b-c61d1cfb0b1c-c000.snappy.parquet new file mode 100644 index 000000000000..5eeb30b719fe Binary files /dev/null and b/presto-delta/src/test/resources/snapshot-data3/part-00001-e62ca5a1-923c-4ee6-998b-c61d1cfb0b1c-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/_delta_log/00000000000000000000.json b/presto-delta/src/test/resources/time-travel-partition-changes-b/_delta_log/00000000000000000000.json new file mode 100644 index 000000000000..192815d32895 --- /dev/null +++ b/presto-delta/src/test/resources/time-travel-partition-changes-b/_delta_log/00000000000000000000.json @@ -0,0 +1,13 @@ +{"commitInfo":{"timestamp":1603724026157,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[\"part5\"]"},"isBlindAppend":true}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"9ce7bb6f-507b-4925-a820-f33601e5d700","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"part5\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["part5"],"configuration":{},"createdTime":1603724025794}} +{"add":{"path":"part5=0/part-00000-67b6882e-f49f-4df5-9850-b5e8a72f4917.c000.snappy.parquet","partitionValues":{"part5":"0"},"size":429,"modificationTime":1603724025000,"dataChange":true}} +{"add":{"path":"part5=1/part-00000-8a40c3d2-f658-4131-a17f-388265ab04b7.c000.snappy.parquet","partitionValues":{"part5":"1"},"size":429,"modificationTime":1603724026000,"dataChange":true}} +{"add":{"path":"part5=2/part-00000-ec6e3a2e-ecbf-4d39-9076-37e523cd62f1.c000.snappy.parquet","partitionValues":{"part5":"2"},"size":429,"modificationTime":1603724026000,"dataChange":true}} +{"add":{"path":"part5=3/part-00000-eaf1edf4-b9da-4df8-b957-08583e2a1d1b.c000.snappy.parquet","partitionValues":{"part5":"3"},"size":429,"modificationTime":1603724026000,"dataChange":true}} +{"add":{"path":"part5=4/part-00000-ce66c2ca-8fdf-48d3-a6e7-5980a370461a.c000.snappy.parquet","partitionValues":{"part5":"4"},"size":429,"modificationTime":1603724026000,"dataChange":true}} +{"add":{"path":"part5=0/part-00001-4f02a740-31dc-46c6-bc0e-c19d164ac82d.c000.snappy.parquet","partitionValues":{"part5":"0"},"size":429,"modificationTime":1603724025000,"dataChange":true}} +{"add":{"path":"part5=1/part-00001-3dcad520-b001-4829-a6e5-3d578b0964f4.c000.snappy.parquet","partitionValues":{"part5":"1"},"size":429,"modificationTime":1603724026000,"dataChange":true}} +{"add":{"path":"part5=2/part-00001-e20bae81-3f27-4c5c-aeca-5cfa6b38615c.c000.snappy.parquet","partitionValues":{"part5":"2"},"size":429,"modificationTime":1603724026000,"dataChange":true}} +{"add":{"path":"part5=3/part-00001-b9c6b926-a274-4d8e-b882-31c4aac05038.c000.snappy.parquet","partitionValues":{"part5":"3"},"size":429,"modificationTime":1603724026000,"dataChange":true}} +{"add":{"path":"part5=4/part-00001-5705917d-d837-4d7f-b8c4-f0ada8cf9663.c000.snappy.parquet","partitionValues":{"part5":"4"},"size":429,"modificationTime":1603724026000,"dataChange":true}} diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/_delta_log/00000000000000000001.json b/presto-delta/src/test/resources/time-travel-partition-changes-b/_delta_log/00000000000000000001.json new file mode 100644 index 000000000000..7cf1e9b774f4 --- /dev/null +++ b/presto-delta/src/test/resources/time-travel-partition-changes-b/_delta_log/00000000000000000001.json @@ -0,0 +1,16 @@ +{"commitInfo":{"timestamp":1603724028432,"operation":"WRITE","operationParameters":{"mode":"Overwrite","partitionBy":"[\"part2\"]"},"readVersion":0,"isBlindAppend":false}} +{"metaData":{"id":"9ce7bb6f-507b-4925-a820-f33601e5d700","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"part2\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["part2"],"configuration":{},"createdTime":1603724025794}} +{"add":{"path":"part2=0/part-00000-7bce012e-f358-4a97-91da-55c4d3266fbe.c000.snappy.parquet","partitionValues":{"part2":"0"},"size":442,"modificationTime":1603724028000,"dataChange":true}} +{"add":{"path":"part2=1/part-00000-82368d1d-588b-487a-be01-16dc85260296.c000.snappy.parquet","partitionValues":{"part2":"1"},"size":437,"modificationTime":1603724028000,"dataChange":true}} +{"add":{"path":"part2=0/part-00001-2a830e69-78f3-4d09-9b2c-3bfd9debc2f0.c000.snappy.parquet","partitionValues":{"part2":"0"},"size":437,"modificationTime":1603724028000,"dataChange":true}} +{"add":{"path":"part2=1/part-00001-0a72544a-fb83-4eaa-8d62-9e6ab59afa8b.c000.snappy.parquet","partitionValues":{"part2":"1"},"size":442,"modificationTime":1603724028000,"dataChange":true}} +{"remove":{"path":"part5=0/part-00000-67b6882e-f49f-4df5-9850-b5e8a72f4917.c000.snappy.parquet","deletionTimestamp":1603724028432,"dataChange":true}} +{"remove":{"path":"part5=0/part-00001-4f02a740-31dc-46c6-bc0e-c19d164ac82d.c000.snappy.parquet","deletionTimestamp":1603724028432,"dataChange":true}} +{"remove":{"path":"part5=1/part-00001-3dcad520-b001-4829-a6e5-3d578b0964f4.c000.snappy.parquet","deletionTimestamp":1603724028432,"dataChange":true}} +{"remove":{"path":"part5=2/part-00000-ec6e3a2e-ecbf-4d39-9076-37e523cd62f1.c000.snappy.parquet","deletionTimestamp":1603724028432,"dataChange":true}} +{"remove":{"path":"part5=2/part-00001-e20bae81-3f27-4c5c-aeca-5cfa6b38615c.c000.snappy.parquet","deletionTimestamp":1603724028432,"dataChange":true}} +{"remove":{"path":"part5=4/part-00000-ce66c2ca-8fdf-48d3-a6e7-5980a370461a.c000.snappy.parquet","deletionTimestamp":1603724028432,"dataChange":true}} +{"remove":{"path":"part5=1/part-00000-8a40c3d2-f658-4131-a17f-388265ab04b7.c000.snappy.parquet","deletionTimestamp":1603724028432,"dataChange":true}} +{"remove":{"path":"part5=3/part-00000-eaf1edf4-b9da-4df8-b957-08583e2a1d1b.c000.snappy.parquet","deletionTimestamp":1603724028432,"dataChange":true}} +{"remove":{"path":"part5=3/part-00001-b9c6b926-a274-4d8e-b882-31c4aac05038.c000.snappy.parquet","deletionTimestamp":1603724028432,"dataChange":true}} +{"remove":{"path":"part5=4/part-00001-5705917d-d837-4d7f-b8c4-f0ada8cf9663.c000.snappy.parquet","deletionTimestamp":1603724028432,"dataChange":true}} diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/part2=0/.part-00000-7bce012e-f358-4a97-91da-55c4d3266fbe.c000.snappy.parquet.crc b/presto-delta/src/test/resources/time-travel-partition-changes-b/part2=0/.part-00000-7bce012e-f358-4a97-91da-55c4d3266fbe.c000.snappy.parquet.crc new file mode 100644 index 000000000000..665b0c6fadab Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-partition-changes-b/part2=0/.part-00000-7bce012e-f358-4a97-91da-55c4d3266fbe.c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/part2=0/.part-00001-2a830e69-78f3-4d09-9b2c-3bfd9debc2f0.c000.snappy.parquet.crc b/presto-delta/src/test/resources/time-travel-partition-changes-b/part2=0/.part-00001-2a830e69-78f3-4d09-9b2c-3bfd9debc2f0.c000.snappy.parquet.crc new file mode 100644 index 000000000000..198c12d32b40 Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-partition-changes-b/part2=0/.part-00001-2a830e69-78f3-4d09-9b2c-3bfd9debc2f0.c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/part2=0/part-00000-7bce012e-f358-4a97-91da-55c4d3266fbe.c000.snappy.parquet b/presto-delta/src/test/resources/time-travel-partition-changes-b/part2=0/part-00000-7bce012e-f358-4a97-91da-55c4d3266fbe.c000.snappy.parquet new file mode 100644 index 000000000000..8b95fb6fe9e9 Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-partition-changes-b/part2=0/part-00000-7bce012e-f358-4a97-91da-55c4d3266fbe.c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/part2=0/part-00001-2a830e69-78f3-4d09-9b2c-3bfd9debc2f0.c000.snappy.parquet b/presto-delta/src/test/resources/time-travel-partition-changes-b/part2=0/part-00001-2a830e69-78f3-4d09-9b2c-3bfd9debc2f0.c000.snappy.parquet new file mode 100644 index 000000000000..1d044a07c4ae Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-partition-changes-b/part2=0/part-00001-2a830e69-78f3-4d09-9b2c-3bfd9debc2f0.c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/part2=1/.part-00000-82368d1d-588b-487a-be01-16dc85260296.c000.snappy.parquet.crc b/presto-delta/src/test/resources/time-travel-partition-changes-b/part2=1/.part-00000-82368d1d-588b-487a-be01-16dc85260296.c000.snappy.parquet.crc new file mode 100644 index 000000000000..39dc81d6349f Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-partition-changes-b/part2=1/.part-00000-82368d1d-588b-487a-be01-16dc85260296.c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/part2=1/.part-00001-0a72544a-fb83-4eaa-8d62-9e6ab59afa8b.c000.snappy.parquet.crc b/presto-delta/src/test/resources/time-travel-partition-changes-b/part2=1/.part-00001-0a72544a-fb83-4eaa-8d62-9e6ab59afa8b.c000.snappy.parquet.crc new file mode 100644 index 000000000000..32ba636ce5f5 Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-partition-changes-b/part2=1/.part-00001-0a72544a-fb83-4eaa-8d62-9e6ab59afa8b.c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/part2=1/part-00000-82368d1d-588b-487a-be01-16dc85260296.c000.snappy.parquet b/presto-delta/src/test/resources/time-travel-partition-changes-b/part2=1/part-00000-82368d1d-588b-487a-be01-16dc85260296.c000.snappy.parquet new file mode 100644 index 000000000000..9f6f4ad0e0c6 Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-partition-changes-b/part2=1/part-00000-82368d1d-588b-487a-be01-16dc85260296.c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/part2=1/part-00001-0a72544a-fb83-4eaa-8d62-9e6ab59afa8b.c000.snappy.parquet b/presto-delta/src/test/resources/time-travel-partition-changes-b/part2=1/part-00001-0a72544a-fb83-4eaa-8d62-9e6ab59afa8b.c000.snappy.parquet new file mode 100644 index 000000000000..33a1e34eaeeb Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-partition-changes-b/part2=1/part-00001-0a72544a-fb83-4eaa-8d62-9e6ab59afa8b.c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=0/.part-00000-67b6882e-f49f-4df5-9850-b5e8a72f4917.c000.snappy.parquet.crc b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=0/.part-00000-67b6882e-f49f-4df5-9850-b5e8a72f4917.c000.snappy.parquet.crc new file mode 100644 index 000000000000..673f94d1016e Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=0/.part-00000-67b6882e-f49f-4df5-9850-b5e8a72f4917.c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=0/.part-00001-4f02a740-31dc-46c6-bc0e-c19d164ac82d.c000.snappy.parquet.crc b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=0/.part-00001-4f02a740-31dc-46c6-bc0e-c19d164ac82d.c000.snappy.parquet.crc new file mode 100644 index 000000000000..3531a2012fae Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=0/.part-00001-4f02a740-31dc-46c6-bc0e-c19d164ac82d.c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=0/part-00000-67b6882e-f49f-4df5-9850-b5e8a72f4917.c000.snappy.parquet b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=0/part-00000-67b6882e-f49f-4df5-9850-b5e8a72f4917.c000.snappy.parquet new file mode 100644 index 000000000000..4cb7aae46305 Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=0/part-00000-67b6882e-f49f-4df5-9850-b5e8a72f4917.c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=0/part-00001-4f02a740-31dc-46c6-bc0e-c19d164ac82d.c000.snappy.parquet b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=0/part-00001-4f02a740-31dc-46c6-bc0e-c19d164ac82d.c000.snappy.parquet new file mode 100644 index 000000000000..2964979ef6db Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=0/part-00001-4f02a740-31dc-46c6-bc0e-c19d164ac82d.c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=1/.part-00000-8a40c3d2-f658-4131-a17f-388265ab04b7.c000.snappy.parquet.crc b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=1/.part-00000-8a40c3d2-f658-4131-a17f-388265ab04b7.c000.snappy.parquet.crc new file mode 100644 index 000000000000..6dee6e123326 Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=1/.part-00000-8a40c3d2-f658-4131-a17f-388265ab04b7.c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=1/.part-00001-3dcad520-b001-4829-a6e5-3d578b0964f4.c000.snappy.parquet.crc b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=1/.part-00001-3dcad520-b001-4829-a6e5-3d578b0964f4.c000.snappy.parquet.crc new file mode 100644 index 000000000000..d55f40cbc051 Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=1/.part-00001-3dcad520-b001-4829-a6e5-3d578b0964f4.c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=1/part-00000-8a40c3d2-f658-4131-a17f-388265ab04b7.c000.snappy.parquet b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=1/part-00000-8a40c3d2-f658-4131-a17f-388265ab04b7.c000.snappy.parquet new file mode 100644 index 000000000000..ebf0901460f2 Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=1/part-00000-8a40c3d2-f658-4131-a17f-388265ab04b7.c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=1/part-00001-3dcad520-b001-4829-a6e5-3d578b0964f4.c000.snappy.parquet b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=1/part-00001-3dcad520-b001-4829-a6e5-3d578b0964f4.c000.snappy.parquet new file mode 100644 index 000000000000..a1caecfbace7 Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=1/part-00001-3dcad520-b001-4829-a6e5-3d578b0964f4.c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=2/.part-00000-ec6e3a2e-ecbf-4d39-9076-37e523cd62f1.c000.snappy.parquet.crc b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=2/.part-00000-ec6e3a2e-ecbf-4d39-9076-37e523cd62f1.c000.snappy.parquet.crc new file mode 100644 index 000000000000..3a4bed33c582 Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=2/.part-00000-ec6e3a2e-ecbf-4d39-9076-37e523cd62f1.c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=2/.part-00001-e20bae81-3f27-4c5c-aeca-5cfa6b38615c.c000.snappy.parquet.crc b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=2/.part-00001-e20bae81-3f27-4c5c-aeca-5cfa6b38615c.c000.snappy.parquet.crc new file mode 100644 index 000000000000..0cd5190c2cd6 Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=2/.part-00001-e20bae81-3f27-4c5c-aeca-5cfa6b38615c.c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=2/part-00000-ec6e3a2e-ecbf-4d39-9076-37e523cd62f1.c000.snappy.parquet b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=2/part-00000-ec6e3a2e-ecbf-4d39-9076-37e523cd62f1.c000.snappy.parquet new file mode 100644 index 000000000000..3b91e5be30c4 Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=2/part-00000-ec6e3a2e-ecbf-4d39-9076-37e523cd62f1.c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=2/part-00001-e20bae81-3f27-4c5c-aeca-5cfa6b38615c.c000.snappy.parquet b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=2/part-00001-e20bae81-3f27-4c5c-aeca-5cfa6b38615c.c000.snappy.parquet new file mode 100644 index 000000000000..1cd670d1938d Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=2/part-00001-e20bae81-3f27-4c5c-aeca-5cfa6b38615c.c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=3/.part-00000-eaf1edf4-b9da-4df8-b957-08583e2a1d1b.c000.snappy.parquet.crc b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=3/.part-00000-eaf1edf4-b9da-4df8-b957-08583e2a1d1b.c000.snappy.parquet.crc new file mode 100644 index 000000000000..3a0fbe2b5ff5 Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=3/.part-00000-eaf1edf4-b9da-4df8-b957-08583e2a1d1b.c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=3/.part-00001-b9c6b926-a274-4d8e-b882-31c4aac05038.c000.snappy.parquet.crc b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=3/.part-00001-b9c6b926-a274-4d8e-b882-31c4aac05038.c000.snappy.parquet.crc new file mode 100644 index 000000000000..aa9bdb761fdb Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=3/.part-00001-b9c6b926-a274-4d8e-b882-31c4aac05038.c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=3/part-00000-eaf1edf4-b9da-4df8-b957-08583e2a1d1b.c000.snappy.parquet b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=3/part-00000-eaf1edf4-b9da-4df8-b957-08583e2a1d1b.c000.snappy.parquet new file mode 100644 index 000000000000..ec63649165cd Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=3/part-00000-eaf1edf4-b9da-4df8-b957-08583e2a1d1b.c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=3/part-00001-b9c6b926-a274-4d8e-b882-31c4aac05038.c000.snappy.parquet b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=3/part-00001-b9c6b926-a274-4d8e-b882-31c4aac05038.c000.snappy.parquet new file mode 100644 index 000000000000..8a4536406db1 Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=3/part-00001-b9c6b926-a274-4d8e-b882-31c4aac05038.c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=4/.part-00000-ce66c2ca-8fdf-48d3-a6e7-5980a370461a.c000.snappy.parquet.crc b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=4/.part-00000-ce66c2ca-8fdf-48d3-a6e7-5980a370461a.c000.snappy.parquet.crc new file mode 100644 index 000000000000..4c990a139198 Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=4/.part-00000-ce66c2ca-8fdf-48d3-a6e7-5980a370461a.c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=4/.part-00001-5705917d-d837-4d7f-b8c4-f0ada8cf9663.c000.snappy.parquet.crc b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=4/.part-00001-5705917d-d837-4d7f-b8c4-f0ada8cf9663.c000.snappy.parquet.crc new file mode 100644 index 000000000000..551c8f5742f5 Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=4/.part-00001-5705917d-d837-4d7f-b8c4-f0ada8cf9663.c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=4/part-00000-ce66c2ca-8fdf-48d3-a6e7-5980a370461a.c000.snappy.parquet b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=4/part-00000-ce66c2ca-8fdf-48d3-a6e7-5980a370461a.c000.snappy.parquet new file mode 100644 index 000000000000..ea2985c2bc2c Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=4/part-00000-ce66c2ca-8fdf-48d3-a6e7-5980a370461a.c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=4/part-00001-5705917d-d837-4d7f-b8c4-f0ada8cf9663.c000.snappy.parquet b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=4/part-00001-5705917d-d837-4d7f-b8c4-f0ada8cf9663.c000.snappy.parquet new file mode 100644 index 000000000000..533801e17f8f Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-partition-changes-b/part5=4/part-00001-5705917d-d837-4d7f-b8c4-f0ada8cf9663.c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/time-travel-schema-changes-b/.part-00000-83680aa8-547c-40bc-8ca9-5c10997e307b-c000.snappy.parquet.crc b/presto-delta/src/test/resources/time-travel-schema-changes-b/.part-00000-83680aa8-547c-40bc-8ca9-5c10997e307b-c000.snappy.parquet.crc new file mode 100644 index 000000000000..919ac1ceb47d Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-schema-changes-b/.part-00000-83680aa8-547c-40bc-8ca9-5c10997e307b-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/time-travel-schema-changes-b/.part-00000-a830a49c-6cc8-4caf-80a5-7ff8a959bd53-c000.snappy.parquet.crc b/presto-delta/src/test/resources/time-travel-schema-changes-b/.part-00000-a830a49c-6cc8-4caf-80a5-7ff8a959bd53-c000.snappy.parquet.crc new file mode 100644 index 000000000000..d586caeb413e Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-schema-changes-b/.part-00000-a830a49c-6cc8-4caf-80a5-7ff8a959bd53-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/time-travel-schema-changes-b/.part-00001-3c1f89ce-a996-4d44-a79c-21a6f3d53138-c000.snappy.parquet.crc b/presto-delta/src/test/resources/time-travel-schema-changes-b/.part-00001-3c1f89ce-a996-4d44-a79c-21a6f3d53138-c000.snappy.parquet.crc new file mode 100644 index 000000000000..03adb57481dc Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-schema-changes-b/.part-00001-3c1f89ce-a996-4d44-a79c-21a6f3d53138-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/time-travel-schema-changes-b/.part-00001-5fdfd303-d5e8-4e77-9b5d-4e831fa723e1-c000.snappy.parquet.crc b/presto-delta/src/test/resources/time-travel-schema-changes-b/.part-00001-5fdfd303-d5e8-4e77-9b5d-4e831fa723e1-c000.snappy.parquet.crc new file mode 100644 index 000000000000..d104b14b1499 Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-schema-changes-b/.part-00001-5fdfd303-d5e8-4e77-9b5d-4e831fa723e1-c000.snappy.parquet.crc differ diff --git a/presto-delta/src/test/resources/time-travel-schema-changes-b/_delta_log/00000000000000000000.json b/presto-delta/src/test/resources/time-travel-schema-changes-b/_delta_log/00000000000000000000.json new file mode 100644 index 000000000000..43ec00a86ab3 --- /dev/null +++ b/presto-delta/src/test/resources/time-travel-schema-changes-b/_delta_log/00000000000000000000.json @@ -0,0 +1,5 @@ +{"commitInfo":{"timestamp":1603724023478,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"isBlindAppend":true}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"37664cd7-239f-4dbc-a56b-d47437be8ddb","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1603724023419}} +{"add":{"path":"part-00000-83680aa8-547c-40bc-8ca9-5c10997e307b-c000.snappy.parquet","partitionValues":{},"size":449,"modificationTime":1603724023000,"dataChange":true}} +{"add":{"path":"part-00001-3c1f89ce-a996-4d44-a79c-21a6f3d53138-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1603724023000,"dataChange":true}} diff --git a/presto-delta/src/test/resources/time-travel-schema-changes-b/_delta_log/00000000000000000001.json b/presto-delta/src/test/resources/time-travel-schema-changes-b/_delta_log/00000000000000000001.json new file mode 100644 index 000000000000..e7ab72b3d822 --- /dev/null +++ b/presto-delta/src/test/resources/time-travel-schema-changes-b/_delta_log/00000000000000000001.json @@ -0,0 +1,4 @@ +{"commitInfo":{"timestamp":1603724024783,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":0,"isBlindAppend":true}} +{"metaData":{"id":"37664cd7-239f-4dbc-a56b-d47437be8ddb","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"part\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1603724023419}} +{"add":{"path":"part-00000-a830a49c-6cc8-4caf-80a5-7ff8a959bd53-c000.snappy.parquet","partitionValues":{},"size":711,"modificationTime":1603724024000,"dataChange":true}} +{"add":{"path":"part-00001-5fdfd303-d5e8-4e77-9b5d-4e831fa723e1-c000.snappy.parquet","partitionValues":{},"size":711,"modificationTime":1603724024000,"dataChange":true}} diff --git a/presto-delta/src/test/resources/time-travel-schema-changes-b/part-00000-83680aa8-547c-40bc-8ca9-5c10997e307b-c000.snappy.parquet b/presto-delta/src/test/resources/time-travel-schema-changes-b/part-00000-83680aa8-547c-40bc-8ca9-5c10997e307b-c000.snappy.parquet new file mode 100644 index 000000000000..f3512340b407 Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-schema-changes-b/part-00000-83680aa8-547c-40bc-8ca9-5c10997e307b-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/time-travel-schema-changes-b/part-00000-a830a49c-6cc8-4caf-80a5-7ff8a959bd53-c000.snappy.parquet b/presto-delta/src/test/resources/time-travel-schema-changes-b/part-00000-a830a49c-6cc8-4caf-80a5-7ff8a959bd53-c000.snappy.parquet new file mode 100644 index 000000000000..5829df4c4776 Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-schema-changes-b/part-00000-a830a49c-6cc8-4caf-80a5-7ff8a959bd53-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/time-travel-schema-changes-b/part-00001-3c1f89ce-a996-4d44-a79c-21a6f3d53138-c000.snappy.parquet b/presto-delta/src/test/resources/time-travel-schema-changes-b/part-00001-3c1f89ce-a996-4d44-a79c-21a6f3d53138-c000.snappy.parquet new file mode 100644 index 000000000000..602abf17284d Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-schema-changes-b/part-00001-3c1f89ce-a996-4d44-a79c-21a6f3d53138-c000.snappy.parquet differ diff --git a/presto-delta/src/test/resources/time-travel-schema-changes-b/part-00001-5fdfd303-d5e8-4e77-9b5d-4e831fa723e1-c000.snappy.parquet b/presto-delta/src/test/resources/time-travel-schema-changes-b/part-00001-5fdfd303-d5e8-4e77-9b5d-4e831fa723e1-c000.snappy.parquet new file mode 100644 index 000000000000..341510ebd9e2 Binary files /dev/null and b/presto-delta/src/test/resources/time-travel-schema-changes-b/part-00001-5fdfd303-d5e8-4e77-9b5d-4e831fa723e1-c000.snappy.parquet differ diff --git a/presto-server-main/etc/catalog/delta.properties b/presto-server-main/etc/catalog/delta.properties new file mode 100644 index 000000000000..92e040f1fb5e --- /dev/null +++ b/presto-server-main/etc/catalog/delta.properties @@ -0,0 +1,2 @@ +connector.name=delta +hive.metastore.uri=thrift://localhost:9083 diff --git a/presto-server-main/etc/config.properties b/presto-server-main/etc/config.properties index c013971c221f..e188ecfef61c 100644 --- a/presto-server-main/etc/config.properties +++ b/presto-server-main/etc/config.properties @@ -48,6 +48,7 @@ plugin.bundles=\ ../presto-thrift/pom.xml, \ ../presto-tpcds/pom.xml, \ ../presto-google-sheets/pom.xml, \ - ../presto-druid/pom.xml + ../presto-druid/pom.xml, \ + ../presto-delta/pom.xml node-scheduler.include-coordinator=true diff --git a/presto-server/src/main/provisio/presto.xml b/presto-server/src/main/provisio/presto.xml index 1ef267f816cc..1ff4d94793b0 100644 --- a/presto-server/src/main/provisio/presto.xml +++ b/presto-server/src/main/provisio/presto.xml @@ -252,4 +252,10 @@ + + + + + +