diff --git a/presto-base-jdbc/src/main/java/com/facebook/presto/plugin/jdbc/optimization/JdbcComputePushdown.java b/presto-base-jdbc/src/main/java/com/facebook/presto/plugin/jdbc/optimization/JdbcComputePushdown.java index 058d7a396e30a..602c3768c68bb 100644 --- a/presto-base-jdbc/src/main/java/com/facebook/presto/plugin/jdbc/optimization/JdbcComputePushdown.java +++ b/presto-base-jdbc/src/main/java/com/facebook/presto/plugin/jdbc/optimization/JdbcComputePushdown.java @@ -156,6 +156,7 @@ public PlanNode visitFilter(FilterNode node, Void context) tableHandle, oldTableScanNode.getOutputVariables(), oldTableScanNode.getAssignments(), + oldTableScanNode.getTableConstraints(), oldTableScanNode.getCurrentConstraint(), oldTableScanNode.getEnforcedConstraint()); diff --git a/presto-druid/src/main/java/com/facebook/presto/druid/DruidPlanOptimizer.java b/presto-druid/src/main/java/com/facebook/presto/druid/DruidPlanOptimizer.java index e0383f973a614..81401f78c8c7b 100644 --- a/presto-druid/src/main/java/com/facebook/presto/druid/DruidPlanOptimizer.java +++ b/presto-druid/src/main/java/com/facebook/presto/druid/DruidPlanOptimizer.java @@ -171,6 +171,7 @@ private Optional tryCreatingNewScanNode(PlanNode plan) newTableHandle, ImmutableList.copyOf(assignments.keySet()), assignments.entrySet().stream().collect(toImmutableMap(Map.Entry::getKey, (e) -> (ColumnHandle) (e.getValue()))), + tableScanNode.getTableConstraints(), tableScanNode.getCurrentConstraint(), tableScanNode.getEnforcedConstraint())); } diff --git a/presto-hive-hadoop2/src/test/java/com/facebook/presto/hive/TestHiveClient.java b/presto-hive-hadoop2/src/test/java/com/facebook/presto/hive/TestHiveClient.java index 80b309c05ddba..0543298e49dbb 100644 --- a/presto-hive-hadoop2/src/test/java/com/facebook/presto/hive/TestHiveClient.java +++ b/presto-hive-hadoop2/src/test/java/com/facebook/presto/hive/TestHiveClient.java @@ -108,4 +108,14 @@ public void testMismatchSchemaTable() super.testMismatchSchemaTable(); } + + @Override + public void testTableConstraints() + { + if (getHiveVersionMajor() < 3) { + throw new SkipException("Table constraints support is in Hive 3 and above. Disabling it for lower versions"); + } + + super.testTableConstraints(); + } } diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/CachingHiveMetastore.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/CachingHiveMetastore.java index 1976068f5cf1d..e483eb805264d 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/CachingHiveMetastore.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/CachingHiveMetastore.java @@ -19,6 +19,7 @@ import com.facebook.presto.hive.HiveType; import com.facebook.presto.hive.MetastoreClientConfig; import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.constraints.TableConstraint; import com.facebook.presto.spi.security.PrestoPrincipal; import com.facebook.presto.spi.security.RoleGrant; import com.facebook.presto.spi.statistics.ColumnStatisticType; @@ -85,11 +86,13 @@ public enum MetastoreCacheScope } protected final ExtendedHiveMetastore delegate; + private final LoadingCache, Optional> databaseCache; private final LoadingCache, List> databaseNamesCache; private final LoadingCache, Optional> tableCache; private final LoadingCache, Optional>> tableNamesCache; private final LoadingCache, PartitionStatistics> tableStatisticsCache; + private final LoadingCache, List>> tableConstraintsCache; private final LoadingCache, PartitionStatistics> partitionStatisticsCache; private final LoadingCache, Optional>> viewNamesCache; private final LoadingCache, Optional> partitionCache; @@ -259,6 +262,9 @@ public Map, PartitionStatistics> loadAll(Iterab tableCache = newCacheBuilder(cacheExpiresAfterWriteMillis, cacheRefreshMills, cacheMaxSize) .build(asyncReloading(CacheLoader.from(this::loadTable), executor)); + tableConstraintsCache = newCacheBuilder(cacheExpiresAfterWriteMillis, cacheRefreshMills, cacheMaxSize) + .build(asyncReloading(CacheLoader.from(this::loadTableConstraints), executor)); + viewNamesCache = newCacheBuilder(cacheExpiresAfterWriteMillis, cacheRefreshMills, cacheMaxSize) .build(asyncReloading(CacheLoader.from(this::loadAllViews), executor)); @@ -304,6 +310,7 @@ public void flushCache() partitionNamesCache.invalidateAll(); databaseCache.invalidateAll(); tableCache.invalidateAll(); + tableConstraintsCache.invalidateAll(); partitionCache.invalidateAll(); partitionFilterCache.invalidateAll(); tablePrivilegesCache.invalidateAll(); @@ -363,6 +370,12 @@ public Optional
getTable(MetastoreContext metastoreContext, String databa return get(tableCache, getCachingKey(metastoreContext, hiveTableName(databaseName, tableName))); } + @Override + public List> getTableConstraints(MetastoreContext metastoreContext, String databaseName, String tableName) + { + return get(tableConstraintsCache, getCachingKey(metastoreContext, hiveTableName(databaseName, tableName))); + } + @Override public Set getSupportedColumnStatistics(MetastoreContext metastoreContext, Type type) { @@ -374,6 +387,11 @@ private Optional
loadTable(KeyAndContext hiveTableName) return delegate.getTable(hiveTableName.getContext(), hiveTableName.getKey().getDatabaseName(), hiveTableName.getKey().getTableName()); } + private List> loadTableConstraints(KeyAndContext hiveTableName) + { + return delegate.getTableConstraints(hiveTableName.getContext(), hiveTableName.getKey().getDatabaseName(), hiveTableName.getKey().getTableName()); + } + @Override public PartitionStatistics getTableStatistics(MetastoreContext metastoreContext, String databaseName, String tableName) { @@ -608,6 +626,10 @@ protected void invalidateTable(String databaseName, String tableName) .filter(hiveTableNameKey -> hiveTableNameKey.getKey().equals(hiveTableName)) .forEach(tableCache::invalidate); + tableConstraintsCache.asMap().keySet().stream() + .filter(hiveTableNameKey -> hiveTableNameKey.getKey().equals(hiveTableName)) + .forEach(tableConstraintsCache::invalidate); + tableNamesCache.asMap().keySet().stream() .filter(tableNameKey -> tableNameKey.getKey().equals(databaseName)) .forEach(tableNamesCache::invalidate); diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/ExtendedHiveMetastore.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/ExtendedHiveMetastore.java index d146d4f15a8a7..f565c7fe4c987 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/ExtendedHiveMetastore.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/ExtendedHiveMetastore.java @@ -17,9 +17,11 @@ import com.facebook.presto.common.predicate.Domain; import com.facebook.presto.common.type.Type; import com.facebook.presto.hive.HiveType; +import com.facebook.presto.spi.constraints.TableConstraint; import com.facebook.presto.spi.security.PrestoPrincipal; import com.facebook.presto.spi.security.RoleGrant; import com.facebook.presto.spi.statistics.ColumnStatisticType; +import com.google.common.collect.ImmutableList; import io.airlift.units.Duration; import java.util.List; @@ -128,4 +130,9 @@ default void unlock(MetastoreContext metastoreContext, long lockId) { throw new NotSupportedException("Unlock is not supported by default"); } + + default List> getTableConstraints(MetastoreContext metastoreContext, String schemaName, String tableName) + { + return ImmutableList.of(); + } } diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/RecordingHiveMetastore.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/RecordingHiveMetastore.java index e3459429095f4..8f09fd5dd346e 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/RecordingHiveMetastore.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/RecordingHiveMetastore.java @@ -20,6 +20,7 @@ import com.facebook.presto.hive.HiveType; import com.facebook.presto.hive.MetastoreClientConfig; import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.constraints.TableConstraint; import com.facebook.presto.spi.security.PrestoPrincipal; import com.facebook.presto.spi.security.RoleGrant; import com.facebook.presto.spi.statistics.ColumnStatisticType; @@ -65,6 +66,7 @@ public class RecordingHiveMetastore private final Cache> databaseCache; private final Cache> tableCache; + private final Cache>> tableConstraintsCache; private final Cache> supportedColumnStatisticsCache; private final Cache tableStatisticsCache; private final Cache, Map> partitionStatisticsCache; @@ -88,6 +90,7 @@ public RecordingHiveMetastore(@ForRecordingHiveMetastore ExtendedHiveMetastore d databaseCache = createCache(metastoreClientConfig); tableCache = createCache(metastoreClientConfig); + tableConstraintsCache = createCache(metastoreClientConfig); supportedColumnStatisticsCache = createCache(metastoreClientConfig); tableStatisticsCache = createCache(metastoreClientConfig); partitionStatisticsCache = createCache(metastoreClientConfig); @@ -115,6 +118,7 @@ void loadRecording() allRoles = recording.getAllRoles(); databaseCache.putAll(toMap(recording.getDatabases())); tableCache.putAll(toMap(recording.getTables())); + tableConstraintsCache.putAll(toMap(recording.getTableConstraints())); supportedColumnStatisticsCache.putAll(toMap(recording.getSupportedColumnStatistics())); tableStatisticsCache.putAll(toMap(recording.getTableStatistics())); partitionStatisticsCache.putAll(toMap(recording.getPartitionStatistics())); @@ -153,6 +157,7 @@ public void writeRecording() allRoles, toPairs(databaseCache), toPairs(tableCache), + toPairs(tableConstraintsCache), toPairs(supportedColumnStatisticsCache), toPairs(tableStatisticsCache), toPairs(partitionStatisticsCache), @@ -206,6 +211,11 @@ public Optional
getTable(MetastoreContext metastoreContext, String databa return loadValue(tableCache, hiveTableName(databaseName, tableName), () -> delegate.getTable(metastoreContext, databaseName, tableName)); } + public List> getTableConstraints(MetastoreContext metastoreContext, String databaseName, String tableName) + { + return loadValue(tableConstraintsCache, hiveTableName(databaseName, tableName), () -> delegate.getTableConstraints(metastoreContext, databaseName, tableName)); + } + @Override public Set getSupportedColumnStatistics(MetastoreContext metastoreContext, Type type) { @@ -508,6 +518,7 @@ public static class Recording private final Optional> allRoles; private final List>> databases; private final List>> tables; + private final List>>> tableConstraints; private final List>> supportedColumnStatistics; private final List> tableStatistics; private final List, Map>> partitionStatistics; @@ -526,6 +537,7 @@ public Recording( @JsonProperty("allRoles") Optional> allRoles, @JsonProperty("databases") List>> databases, @JsonProperty("tables") List>> tables, + @JsonProperty("tableConstraints") List>>> tableConstraints, @JsonProperty("supportedColumnStatistics") List>> supportedColumnStatistics, @JsonProperty("tableStatistics") List> tableStatistics, @JsonProperty("partitionStatistics") List, Map>> partitionStatistics, @@ -542,6 +554,7 @@ public Recording( this.allRoles = allRoles; this.databases = databases; this.tables = tables; + this.tableConstraints = tableConstraints; this.supportedColumnStatistics = supportedColumnStatistics; this.tableStatistics = tableStatistics; this.partitionStatistics = partitionStatistics; @@ -579,6 +592,12 @@ public List>> getTables() return tables; } + @JsonProperty + public List>>> getTableConstraints() + { + return tableConstraints; + } + @JsonProperty public List>> getSupportedColumnStatistics() { diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/SemiTransactionalHiveMetastore.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/SemiTransactionalHiveMetastore.java index f8a0f5b712b18..66ad1b5cf9cdd 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/SemiTransactionalHiveMetastore.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/SemiTransactionalHiveMetastore.java @@ -31,6 +31,7 @@ import com.facebook.presto.spi.StandardErrorCode; import com.facebook.presto.spi.TableNotFoundException; import com.facebook.presto.spi.connector.ConnectorCommitHandle; +import com.facebook.presto.spi.constraints.TableConstraint; import com.facebook.presto.spi.security.PrestoPrincipal; import com.facebook.presto.spi.security.PrincipalType; import com.facebook.presto.spi.security.RoleGrant; @@ -56,6 +57,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -198,6 +200,16 @@ public synchronized Optional
getTable(MetastoreContext metastoreContext, } } + public synchronized List> getTableConstraints(MetastoreContext metastoreContext, String databaseName, String tableName) + { + checkReadable(); + Action tableAction = tableActions.get(new SchemaTableName(databaseName, tableName)); + if (tableAction == null) { + return delegate.getTableConstraints(metastoreContext, databaseName, tableName); + } + return Collections.emptyList(); + } + public synchronized Set getSupportedColumnStatistics(MetastoreContext metastoreContext, Type type) { return delegate.getSupportedColumnStatistics(metastoreContext, type); diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/BridgingHiveMetastore.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/BridgingHiveMetastore.java index e4a1db050750d..c2f7b48ef0bc8 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/BridgingHiveMetastore.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/BridgingHiveMetastore.java @@ -34,9 +34,12 @@ import com.facebook.presto.spi.SchemaNotFoundException; import com.facebook.presto.spi.SchemaTableName; import com.facebook.presto.spi.TableNotFoundException; +import com.facebook.presto.spi.constraints.PrimaryKeyConstraint; +import com.facebook.presto.spi.constraints.TableConstraint; import com.facebook.presto.spi.security.PrestoPrincipal; import com.facebook.presto.spi.security.RoleGrant; import com.facebook.presto.spi.statistics.ColumnStatisticType; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.airlift.units.Duration; import org.apache.hadoop.hive.metastore.api.FieldSchema; @@ -99,6 +102,18 @@ public Optional
getTable(MetastoreContext metastoreContext, String databa }); } + @Override + public List> getTableConstraints(MetastoreContext metastoreContext, String databaseName, String tableName) + { + ImmutableList.Builder> constraints = ImmutableList.builder(); + Optional> primaryKey = delegate.getPrimaryKey(metastoreContext, databaseName, tableName); + if (primaryKey.isPresent()) { + constraints.add(primaryKey.get()); + } + constraints.addAll(delegate.getUniqueConstraints(metastoreContext, databaseName, tableName)); + return constraints.build(); + } + @Override public Set getSupportedColumnStatistics(MetastoreContext metastoreContext, Type type) { diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/HiveMetastore.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/HiveMetastore.java index 92efc9842e22e..ab5b50927ca37 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/HiveMetastore.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/HiveMetastore.java @@ -25,9 +25,12 @@ import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.SchemaTableName; import com.facebook.presto.spi.TableNotFoundException; +import com.facebook.presto.spi.constraints.PrimaryKeyConstraint; +import com.facebook.presto.spi.constraints.UniqueConstraint; import com.facebook.presto.spi.security.PrestoPrincipal; import com.facebook.presto.spi.security.RoleGrant; import com.facebook.presto.spi.statistics.ColumnStatisticType; +import com.google.common.collect.ImmutableList; import io.airlift.units.Duration; import org.apache.hadoop.hive.metastore.api.Database; import org.apache.hadoop.hive.metastore.api.FieldSchema; @@ -169,4 +172,14 @@ default Optional> getFields(MetastoreContext metastoreContext, return Optional.of(table.get().getSd().getCols()); } + + default Optional> getPrimaryKey(MetastoreContext metastoreContext, String databaseName, String tableName) + { + return Optional.empty(); + } + + default List> getUniqueConstraints(MetastoreContext metastoreContext, String databaseName, String tableName) + { + return ImmutableList.of(); + } } diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/HiveMetastoreClient.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/HiveMetastoreClient.java index e625cc0966dcd..64d7c5af9eb99 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/HiveMetastoreClient.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/HiveMetastoreClient.java @@ -22,17 +22,20 @@ import org.apache.hadoop.hive.metastore.api.LockRequest; import org.apache.hadoop.hive.metastore.api.LockResponse; import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.PrimaryKeysResponse; import org.apache.hadoop.hive.metastore.api.PrincipalType; import org.apache.hadoop.hive.metastore.api.PrivilegeBag; import org.apache.hadoop.hive.metastore.api.Role; import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant; import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.api.UniqueConstraintsResponse; import org.apache.hadoop.hive.metastore.api.UnlockRequest; import org.apache.thrift.TException; import java.io.Closeable; import java.util.List; import java.util.Map; +import java.util.Optional; public interface HiveMetastoreClient extends Closeable @@ -159,4 +162,10 @@ LockResponse lock(LockRequest request) void unlock(UnlockRequest request) throws TException; + + Optional getPrimaryKey(String dbName, String tableName) + throws TException; + + Optional getUniqueConstraints(String catName, String dbName, String tableName) + throws TException; } diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/ThriftHiveMetastore.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/ThriftHiveMetastore.java index 7e1c50a0dde67..c26dc9cd63096 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/ThriftHiveMetastore.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/ThriftHiveMetastore.java @@ -40,6 +40,8 @@ import com.facebook.presto.spi.SchemaNotFoundException; import com.facebook.presto.spi.SchemaTableName; import com.facebook.presto.spi.TableNotFoundException; +import com.facebook.presto.spi.constraints.PrimaryKeyConstraint; +import com.facebook.presto.spi.constraints.UniqueConstraint; import com.facebook.presto.spi.security.ConnectorIdentity; import com.facebook.presto.spi.security.PrestoPrincipal; import com.facebook.presto.spi.security.RoleGrant; @@ -69,10 +71,14 @@ import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.PrimaryKeysResponse; import org.apache.hadoop.hive.metastore.api.PrincipalType; import org.apache.hadoop.hive.metastore.api.PrivilegeBag; import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo; +import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey; +import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint; import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.api.UniqueConstraintsResponse; import org.apache.hadoop.hive.metastore.api.UnknownDBException; import org.apache.hadoop.hive.metastore.api.UnknownTableException; import org.apache.hadoop.hive.metastore.api.UnlockRequest; @@ -138,6 +144,7 @@ import static com.google.common.base.Throwables.throwIfUnchecked; import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableMap.toImmutableMap; +import static com.google.common.collect.ImmutableSet.toImmutableSet; import static com.google.common.collect.Iterables.getOnlyElement; import static com.google.common.collect.Sets.difference; import static java.lang.String.format; @@ -209,6 +216,69 @@ public ThriftHiveMetastoreStats getStats() return stats; } + public Optional> getPrimaryKey(MetastoreContext metastoreContext, String dbName, String tableName) + { + try { + Optional pkResponse = retry() + .stopOnIllegalExceptions() + .run("getPrimaryKey", stats.getGetPrimaryKey().wrap(() -> + getMetastoreClientThenCall(metastoreContext, client -> client.getPrimaryKey(dbName, tableName)))); + + if (!pkResponse.isPresent() || pkResponse.get().getPrimaryKeys().size() == 0) { + return Optional.empty(); + } + + List pkCols = pkResponse.get().getPrimaryKeys(); + boolean isEnabled = pkCols.get(0).isEnable_cstr(); + boolean isRely = pkCols.get(0).isRely_cstr(); + String pkName = pkCols.get(0).getPk_name(); + Set keyCols = pkCols.stream().map(SQLPrimaryKey::getColumn_name).collect(toImmutableSet()); + return Optional.of(new PrimaryKeyConstraint<>(pkName, keyCols, isEnabled, isRely)); + } + catch (TException e) { + throw new PrestoException(HIVE_METASTORE_ERROR, e); + } + catch (Exception e) { + throw propagate(e); + } + } + + @Override + public List> getUniqueConstraints(MetastoreContext metastoreContext, String dbName, String tableName) + { + try { + Optional uniqueConstraintsResponse = retry() + .stopOnIllegalExceptions() + .run("getUniqueConstraints", stats.getGetUniqueConstraints().wrap(() -> + getMetastoreClientThenCall(metastoreContext, client -> client.getUniqueConstraints("hive", dbName, tableName)))); + + if (!uniqueConstraintsResponse.isPresent() || uniqueConstraintsResponse.get().getUniqueConstraints().size() == 0) { + return ImmutableList.of(); + } + + List uniqueConstraints = uniqueConstraintsResponse.get().getUniqueConstraints(); + //bucket the unique constraint columns by constraint name + Map> bucketedConstraints = uniqueConstraints.stream().collect(Collectors.groupingBy(SQLUniqueConstraint::getUk_name)); + + //create a unique table constraint per bucket + ImmutableList> result = bucketedConstraints.entrySet().stream().map(e -> { + String constraintName = e.getKey(); + Set columnNames = e.getValue().stream().map(SQLUniqueConstraint::getColumn_name).collect(toImmutableSet()); + boolean isEnabled = e.getValue().get(0).isEnable_cstr(); + boolean isRely = e.getValue().get(0).isRely_cstr(); + return new UniqueConstraint<>(constraintName, columnNames, isEnabled, isRely); + }).collect(toImmutableList()); + + return result; + } + catch (TException e) { + throw new PrestoException(HIVE_METASTORE_ERROR, e); + } + catch (Exception e) { + throw propagate(e); + } + } + @Override public List getAllDatabases(MetastoreContext context) { diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/ThriftHiveMetastoreClient.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/ThriftHiveMetastoreClient.java index c9c481f758e79..4e9faa8c5a23a 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/ThriftHiveMetastoreClient.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/ThriftHiveMetastoreClient.java @@ -32,6 +32,8 @@ import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.PartitionsStatsRequest; +import org.apache.hadoop.hive.metastore.api.PrimaryKeysRequest; +import org.apache.hadoop.hive.metastore.api.PrimaryKeysResponse; import org.apache.hadoop.hive.metastore.api.PrincipalType; import org.apache.hadoop.hive.metastore.api.PrivilegeBag; import org.apache.hadoop.hive.metastore.api.Role; @@ -39,7 +41,10 @@ import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.api.TableStatsRequest; import org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore; +import org.apache.hadoop.hive.metastore.api.UniqueConstraintsRequest; +import org.apache.hadoop.hive.metastore.api.UniqueConstraintsResponse; import org.apache.hadoop.hive.metastore.api.UnlockRequest; +import org.apache.thrift.TApplicationException; import org.apache.thrift.TException; import org.apache.thrift.protocol.TBinaryProtocol; import org.apache.thrift.protocol.TProtocol; @@ -48,8 +53,10 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Optional; import static java.util.Objects.requireNonNull; +import static org.apache.thrift.TApplicationException.UNKNOWN_METHOD; public class ThriftHiveMetastoreClient implements HiveMetastoreClient @@ -425,4 +432,45 @@ public void unlock(UnlockRequest request) { client.unlock(request); } + + public Optional getPrimaryKey(String dbName, String tableName) + throws TException + { + PrimaryKeysRequest pkRequest = new PrimaryKeysRequest(dbName, tableName); + PrimaryKeysResponse pkResponse; + + try { + pkResponse = client.get_primary_keys(pkRequest); + } + catch (TApplicationException e) { + // If we are talking to Hive version < 3 which doesn't support table constraints, + // then we will get an UNKNOWN_METHOD error. + if (e.getType() == UNKNOWN_METHOD) { + return Optional.empty(); + } + throw e; + } + + return Optional.of(pkResponse); + } + + @Override + public Optional getUniqueConstraints(String catName, String dbName, String tableName) + throws TException + { + UniqueConstraintsRequest uniqueConstraintsRequest = new UniqueConstraintsRequest(catName, dbName, tableName); + UniqueConstraintsResponse uniqueConstraintsResponse; + + try { + uniqueConstraintsResponse = client.get_unique_constraints(uniqueConstraintsRequest); + } + catch (TApplicationException e) { + if (e.getType() == UNKNOWN_METHOD) { + return Optional.empty(); + } + throw e; + } + + return Optional.of(uniqueConstraintsResponse); + } } diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/ThriftHiveMetastoreStats.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/ThriftHiveMetastoreStats.java index 89d2d4414da17..ad9c555622410 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/ThriftHiveMetastoreStats.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/ThriftHiveMetastoreStats.java @@ -52,6 +52,8 @@ public class ThriftHiveMetastoreStats private final HiveMetastoreApiStats dropRole = new HiveMetastoreApiStats(); private final HiveMetastoreApiStats lock = new HiveMetastoreApiStats(); private final HiveMetastoreApiStats unlock = new HiveMetastoreApiStats(); + private final HiveMetastoreApiStats getPrimaryKey = new HiveMetastoreApiStats(); + private final HiveMetastoreApiStats getUniqueConstraints = new HiveMetastoreApiStats(); @Managed @Nested @@ -284,10 +286,20 @@ public HiveMetastoreApiStats getLock() return lock; } + public HiveMetastoreApiStats getGetPrimaryKey() + { + return getPrimaryKey; + } + @Managed @Nested public HiveMetastoreApiStats getUnlock() { return unlock; } + + public HiveMetastoreApiStats getGetUniqueConstraints() + { + return getUniqueConstraints; + } } diff --git a/presto-hive-metastore/src/test/java/com/facebook/presto/hive/metastore/TestRecordingHiveMetastore.java b/presto-hive-metastore/src/test/java/com/facebook/presto/hive/metastore/TestRecordingHiveMetastore.java index 052c63ed49034..66f7b97d1c22b 100644 --- a/presto-hive-metastore/src/test/java/com/facebook/presto/hive/metastore/TestRecordingHiveMetastore.java +++ b/presto-hive-metastore/src/test/java/com/facebook/presto/hive/metastore/TestRecordingHiveMetastore.java @@ -21,6 +21,9 @@ import com.facebook.presto.hive.MetastoreClientConfig; import com.facebook.presto.hive.metastore.HivePrivilegeInfo.HivePrivilege; import com.facebook.presto.hive.metastore.SortingColumn.Order; +import com.facebook.presto.spi.constraints.PrimaryKeyConstraint; +import com.facebook.presto.spi.constraints.TableConstraint; +import com.facebook.presto.spi.constraints.UniqueConstraint; import com.facebook.presto.spi.security.PrestoPrincipal; import com.facebook.presto.spi.security.RoleGrant; import com.facebook.presto.spi.statistics.ColumnStatisticType; @@ -32,6 +35,7 @@ import java.io.File; import java.io.IOException; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -67,6 +71,16 @@ public class TestRecordingHiveMetastore HiveType.HIVE_INT, Optional.of("comment"), Optional.empty()); + private static final Column TABLE_COLUMN_PK = new Column( + "column_pk", + HiveType.HIVE_INT, + Optional.of("Primary Key"), + Optional.empty()); + private static final Column TABLE_COLUMN_UNIQUE = new Column( + "column_unique", + HiveType.HIVE_INT, + Optional.of("Unique Key"), + Optional.empty()); private static final Storage TABLE_STORAGE = new Storage( StorageFormat.create("serde", "input", "output"), "location", @@ -85,7 +99,7 @@ public class TestRecordingHiveMetastore "owner", OTHER, TABLE_STORAGE, - ImmutableList.of(TABLE_COLUMN), + ImmutableList.of(TABLE_COLUMN, TABLE_COLUMN_PK, TABLE_COLUMN_UNIQUE), ImmutableList.of(TABLE_COLUMN), ImmutableMap.of("param", "value3"), Optional.of("original_text"), @@ -117,6 +131,8 @@ public class TestRecordingHiveMetastore OptionalLong.of(8)))); private static final HivePrivilegeInfo PRIVILEGE_INFO = new HivePrivilegeInfo(HivePrivilege.SELECT, true, new PrestoPrincipal(USER, "grantor"), new PrestoPrincipal(USER, "grantee")); private static final RoleGrant ROLE_GRANT = new RoleGrant(new PrestoPrincipal(USER, "grantee"), "role", true); + private static final PrimaryKeyConstraint TEST_PRIMARY_KEY = new PrimaryKeyConstraint<>("", ImmutableSet.of("column_pk"), true, true); + private static final UniqueConstraint TEST_UNIQUE_CONSTRAINT = new UniqueConstraint<>("", ImmutableSet.of("column_unique"), true, true); @Test public void testRecordingHiveMetastore() @@ -159,6 +175,7 @@ private void validateMetadata(ExtendedHiveMetastore hiveMetastore) assertEquals(hiveMetastore.listTablePrivileges(TEST_METASTORE_CONTEXT, "database", "table", new PrestoPrincipal(USER, "user")), ImmutableSet.of(PRIVILEGE_INFO)); assertEquals(hiveMetastore.listRoles(TEST_METASTORE_CONTEXT), ImmutableSet.of("role")); assertEquals(hiveMetastore.listRoleGrants(TEST_METASTORE_CONTEXT, new PrestoPrincipal(USER, "user")), ImmutableSet.of(ROLE_GRANT)); + assertEquals(hiveMetastore.getTableConstraints(TEST_METASTORE_CONTEXT, "database", "table"), ImmutableList.of(TEST_PRIMARY_KEY, TEST_UNIQUE_CONSTRAINT)); } private static class TestingHiveMetastore @@ -308,5 +325,18 @@ public Set listRoleGrants(MetastoreContext metastoreContext, PrestoPr { return ImmutableSet.of(ROLE_GRANT); } + + @Override + public List> getTableConstraints(MetastoreContext metastoreContext, String database, String table) + { + if (database.equals("database") && table.equals("table")) { + List> constraints = new ArrayList<>(); + constraints.add(TEST_PRIMARY_KEY); + constraints.add(TEST_UNIQUE_CONSTRAINT); + return constraints; + } + + return ImmutableList.of(); + } } } diff --git a/presto-hive-metastore/src/test/java/com/facebook/presto/hive/metastore/thrift/MockHiveMetastoreClient.java b/presto-hive-metastore/src/test/java/com/facebook/presto/hive/metastore/thrift/MockHiveMetastoreClient.java index c7b82a12e123f..a3ebae95fa71a 100644 --- a/presto-hive-metastore/src/test/java/com/facebook/presto/hive/metastore/thrift/MockHiveMetastoreClient.java +++ b/presto-hive-metastore/src/test/java/com/facebook/presto/hive/metastore/thrift/MockHiveMetastoreClient.java @@ -35,13 +35,17 @@ import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.PrimaryKeysResponse; import org.apache.hadoop.hive.metastore.api.PrincipalType; import org.apache.hadoop.hive.metastore.api.PrivilegeBag; import org.apache.hadoop.hive.metastore.api.Role; import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant; +import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey; +import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint; import org.apache.hadoop.hive.metastore.api.SerDeInfo; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.api.UniqueConstraintsResponse; import org.apache.hadoop.hive.metastore.api.UnlockRequest; import org.apache.thrift.TException; @@ -59,6 +63,12 @@ public class MockHiveMetastoreClient public static final String TEST_DATABASE = "testdb"; public static final String BAD_DATABASE = "baddb"; public static final String TEST_TABLE = "testtbl"; + public static final String TEST_TABLE_WITH_CONSTRAINTS = "testtbl_constraints"; + public static final Map> SCHEMA_MAP = ImmutableMap.of( + TEST_DATABASE + TEST_TABLE, ImmutableList.of(new FieldSchema("key", "string", null)), + TEST_DATABASE + TEST_TABLE_WITH_CONSTRAINTS, ImmutableList.of(new FieldSchema("c1", "string", "Primary Key"), new FieldSchema("c2", "string", "Unique Key"))); + public static final List TEST_PRIMARY_KEY = ImmutableList.of(new SQLPrimaryKey(TEST_DATABASE, TEST_TABLE_WITH_CONSTRAINTS, "c1", 0, "", true, false, true)); + public static final List TEST_UNIQUE_CONSTRAINT = ImmutableList.of(new SQLUniqueConstraint("", TEST_DATABASE, TEST_TABLE_WITH_CONSTRAINTS, "c2", 1, "", true, false, true)); public static final String TEST_TOKEN = "token"; public static final MetastoreContext TEST_METASTORE_CONTEXT = new MetastoreContext("test_user", "test_queryId", Optional.empty(), Optional.empty(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); public static final String TEST_PARTITION1 = "key=testpartition1"; @@ -127,7 +137,7 @@ public List getAllTables(String dbName) if (!dbName.equals(TEST_DATABASE)) { return ImmutableList.of(); // As specified by Hive specification } - return ImmutableList.of(TEST_TABLE); + return ImmutableList.of(TEST_TABLE, TEST_TABLE_WITH_CONSTRAINTS); } @Override @@ -152,9 +162,10 @@ public Table getTable(String dbName, String tableName) if (throwException) { throw new RuntimeException(); } - if (!dbName.equals(TEST_DATABASE) || !tableName.equals(TEST_TABLE)) { + if (!dbName.equals(TEST_DATABASE) || (!tableName.equals(TEST_TABLE) && !tableName.equals(TEST_TABLE_WITH_CONSTRAINTS))) { throw new NoSuchObjectException(); } + return new Table( TEST_TABLE, TEST_DATABASE, @@ -163,7 +174,7 @@ public Table getTable(String dbName, String tableName) 0, 0, DEFAULT_STORAGE_DESCRIPTOR, - ImmutableList.of(new FieldSchema("key", "string", null)), + SCHEMA_MAP.get(dbName + tableName), null, "", "", @@ -450,4 +461,23 @@ public void unlock(UnlockRequest request) { throw new UnsupportedOperationException(); } + + public Optional getPrimaryKey(String dbName, String tableName) + { + accessCount.incrementAndGet(); + if (!dbName.equals(TEST_DATABASE) || !tableName.equals(TEST_TABLE_WITH_CONSTRAINTS)) { + throw new UnsupportedOperationException(); + } + return Optional.of(new PrimaryKeysResponse(TEST_PRIMARY_KEY)); + } + + @Override + public Optional getUniqueConstraints(String catName, String dbName, String tableName) + { + accessCount.incrementAndGet(); + if (!dbName.equals(TEST_DATABASE) || !tableName.equals(TEST_TABLE_WITH_CONSTRAINTS)) { + throw new UnsupportedOperationException(); + } + return Optional.of(new UniqueConstraintsResponse(TEST_UNIQUE_CONSTRAINT)); + } } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java index 481464777751e..d1b415665bc2c 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java @@ -81,6 +81,7 @@ import com.facebook.presto.spi.connector.ConnectorPartitioningHandle; import com.facebook.presto.spi.connector.ConnectorPartitioningMetadata; import com.facebook.presto.spi.connector.ConnectorTransactionHandle; +import com.facebook.presto.spi.constraints.TableConstraint; import com.facebook.presto.spi.function.StandardFunctionResolution; import com.facebook.presto.spi.plan.FilterStatsCalculatorService; import com.facebook.presto.spi.relation.RowExpression; @@ -134,6 +135,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Comparator; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -634,8 +636,18 @@ private ConnectorTableMetadata getTableMetadata(ConnectorSession session, Schema Function metadataGetter = columnMetadataGetter(table.get(), typeManager, metastoreContext.getColumnConverter()); ImmutableList.Builder columns = ImmutableList.builder(); + Map columnNameToHandleAssignments = new HashMap<>(); for (HiveColumnHandle columnHandle : hiveColumnHandles(table.get())) { columns.add(metadataGetter.apply(columnHandle)); + columnNameToHandleAssignments.put(columnHandle.getName(), columnHandle); + } + + List> tableConstraints = ImmutableList.of(); + + if (session.isReadConstraints()) { + // Get table constraints and rebase on column handles from column names + List> metastoreTableConstraints = metastore.getTableConstraints(metastoreContext, tableName.getSchemaName(), tableName.getTableName()); + tableConstraints = ConnectorTableMetadata.rebaseTableConstraints(metastoreTableConstraints, columnNameToHandleAssignments); } // External location property @@ -710,7 +722,7 @@ private ConnectorTableMetadata getTableMetadata(ConnectorSession session, Schema Optional comment = Optional.ofNullable(table.get().getParameters().get(TABLE_COMMENT)); - return new ConnectorTableMetadata(tableName, columns.build(), properties.build(), comment); + return new ConnectorTableMetadata(tableName, columns.build(), properties.build(), comment, tableConstraints); } private static Optional getCsvSerdeProperty(Table table, String key) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HivePartialAggregationPushdown.java b/presto-hive/src/main/java/com/facebook/presto/hive/HivePartialAggregationPushdown.java index 7712ae9dab1ed..ae84f066b7ded 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HivePartialAggregationPushdown.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HivePartialAggregationPushdown.java @@ -278,6 +278,7 @@ private Optional tryPartialAggregationPushdown(PlanNode plan) newTableHandle, ImmutableList.copyOf(partialAggregationNode.getOutputVariables()), ImmutableMap.copyOf(assignments), + oldTableScanNode.getTableConstraints(), oldTableScanNode.getCurrentConstraint(), oldTableScanNode.getEnforcedConstraint())); } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveSessionProperties.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveSessionProperties.java index b16d9497aec58..5d40b8da4c060 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveSessionProperties.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveSessionProperties.java @@ -139,6 +139,7 @@ public final class HiveSessionProperties public static final String MAX_INITIAL_SPLITS = "max_initial_splits"; public static final String FILE_SPLITTABLE = "file_splittable"; private static final String HUDI_METADATA_ENABLED = "hudi_metadata_enabled"; + private static final String READ_TABLE_CONSTRAINTS = "read_table_constraints"; private final List> sessionProperties; @@ -1170,4 +1171,9 @@ public static boolean isHudiMetadataEnabled(ConnectorSession session) { return session.getProperty(HUDI_METADATA_ENABLED, Boolean.class); } + + public static boolean isReadTableConstraints(ConnectorSession session) + { + return session.getProperty(READ_TABLE_CONSTRAINTS, Boolean.class); + } } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/rule/HiveAddRequestedColumnsToLayout.java b/presto-hive/src/main/java/com/facebook/presto/hive/rule/HiveAddRequestedColumnsToLayout.java index 5e7a8e58d717d..ee7b53614e3b6 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/rule/HiveAddRequestedColumnsToLayout.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/rule/HiveAddRequestedColumnsToLayout.java @@ -103,6 +103,7 @@ public PlanNode visitTableScan(TableScanNode tableScan, Void context) Optional.of(hiveLayoutWithDesiredColumns)), tableScan.getOutputVariables(), tableScan.getAssignments(), + tableScan.getTableConstraints(), tableScan.getCurrentConstraint(), tableScan.getEnforcedConstraint()); } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/rule/HiveFilterPushdown.java b/presto-hive/src/main/java/com/facebook/presto/hive/rule/HiveFilterPushdown.java index da680c663bf8a..351d47dea4e04 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/rule/HiveFilterPushdown.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/rule/HiveFilterPushdown.java @@ -396,6 +396,7 @@ public PlanNode visitFilter(FilterNode filter, Void context) new TableHandle(handle.getConnectorId(), handle.getConnectorHandle(), handle.getTransaction(), Optional.of(pushdownFilterResult.getLayout().getHandle())), tableScan.getOutputVariables(), tableScan.getAssignments(), + tableScan.getTableConstraints(), layout.getPredicate(), TupleDomain.all()); @@ -431,6 +432,7 @@ public PlanNode visitTableScan(TableScanNode tableScan, Void context) new TableHandle(handle.getConnectorId(), handle.getConnectorHandle(), handle.getTransaction(), Optional.of(pushdownFilterResult.getLayout().getHandle())), tableScan.getOutputVariables(), tableScan.getAssignments(), + tableScan.getTableConstraints(), pushdownFilterResult.getLayout().getPredicate(), TupleDomain.all()); } diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java b/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java index 906a4914c2ad6..b79348775cbad 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java @@ -102,6 +102,9 @@ import com.facebook.presto.spi.connector.ConnectorSplitManager; import com.facebook.presto.spi.connector.ConnectorSplitManager.SplitSchedulingContext; import com.facebook.presto.spi.connector.ConnectorTransactionHandle; +import com.facebook.presto.spi.constraints.PrimaryKeyConstraint; +import com.facebook.presto.spi.constraints.TableConstraint; +import com.facebook.presto.spi.constraints.UniqueConstraint; import com.facebook.presto.spi.function.SqlFunctionId; import com.facebook.presto.spi.function.SqlInvokedFunction; import com.facebook.presto.spi.relation.RowExpression; @@ -711,6 +714,23 @@ private static RowExpression toRowExpression(String sql) protected SchemaTableName tablePartitionSchemaChange; protected SchemaTableName tablePartitionSchemaChangeNonCanonical; protected SchemaTableName tableBucketEvolution; + protected SchemaTableName tableConstraintsSingleKeyRely; + protected SchemaTableName tableConstraintsMultiKeyRely; + protected SchemaTableName tableConstraintsSingleKeyNoRely; + protected SchemaTableName tableConstraintsMultiKeyNoRely; + + protected List constraintsTableList; + + private static final List> constraintsSingleKeyRely = ImmutableList.of(new PrimaryKeyConstraint<>("", ImmutableSet.of("c1"), false, true), + new UniqueConstraint<>("uk1", ImmutableSet.of("c2"), false, true)); + private static final List> constraintsMultiKeyRely = ImmutableList.of(new PrimaryKeyConstraint<>("", ImmutableSet.of("c1", "c2"), false, true), + new UniqueConstraint<>("uk2", ImmutableSet.of("c3", "c4"), false, true)); + private static final List> constraintsSingleKeyNoRely = ImmutableList.of(new PrimaryKeyConstraint<>("", ImmutableSet.of("c1"), false, false), + new UniqueConstraint<>("uk3", ImmutableSet.of("c2"), false, false)); + private static final List> constraintsMultiKeyNoRely = ImmutableList.of(new PrimaryKeyConstraint<>("", ImmutableSet.of("c1", "c2"), false, false), + new UniqueConstraint<>("uk4", ImmutableSet.of("c3", "c4"), false, false)); + + Map>> tableConstraintsMap; protected String invalidClientId; protected ConnectorTableHandle invalidTableHandle; @@ -774,6 +794,20 @@ protected void setupHive(String connectorId, String databaseName, String timeZon tablePartitionSchemaChange = new SchemaTableName(database, "presto_test_partition_schema_change"); tablePartitionSchemaChangeNonCanonical = new SchemaTableName(database, "presto_test_partition_schema_change_non_canonical"); tableBucketEvolution = new SchemaTableName(database, "presto_test_bucket_evolution"); + tableConstraintsSingleKeyRely = new SchemaTableName(database, "test_constraints1"); + tableConstraintsMultiKeyRely = new SchemaTableName(database, "test_constraints2"); + tableConstraintsSingleKeyNoRely = new SchemaTableName(database, "test_constraints3"); + tableConstraintsMultiKeyNoRely = new SchemaTableName(database, "test_constraints4"); + constraintsTableList = ImmutableList.of(tableConstraintsSingleKeyRely, + tableConstraintsMultiKeyRely, + tableConstraintsSingleKeyNoRely, + tableConstraintsMultiKeyNoRely, + tablePartitionFormat); + tableConstraintsMap = ImmutableMap.of(tableConstraintsSingleKeyRely, constraintsSingleKeyRely, + tableConstraintsMultiKeyRely, constraintsMultiKeyRely, + tableConstraintsSingleKeyNoRely, constraintsSingleKeyNoRely, + tableConstraintsMultiKeyNoRely, constraintsMultiKeyNoRely, + tablePartitionFormat, ImmutableList.of()); invalidClientId = "hive"; invalidTableHandle = new HiveTableHandle(database, INVALID_TABLE); @@ -5640,6 +5674,42 @@ private void doTestTransactionDeleteInsert( } } + @Test + public void testTableConstraints() + { + for (SchemaTableName table : constraintsTableList) { + List> tableConstraints = getTableConstraints(table); + List> expectedConstraints = tableConstraintsMap.get(table); + compareTableConstraints(tableConstraints, expectedConstraints); + } + } + + private List> getTableConstraints(SchemaTableName tableName) + { + ConnectorSession session = newSession(); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + return metastoreClient.getTableConstraints(metastoreContext, tableName.getSchemaName(), tableName.getTableName()); + } + + private void compareTableConstraints(List> tableConstraints, List> expectedConstraints) + { + assertEquals(tableConstraints.size(), expectedConstraints.size()); + + for (int i = 0; i < tableConstraints.size(); i++) { + TableConstraint constraint = tableConstraints.get(i); + TableConstraint expectedConstraint = expectedConstraints.get(i); + // Hive primary key name is auto-generated, hence explicit comparison of members excluding name + if (constraint instanceof PrimaryKeyConstraint) { + assertEquals(constraint.getColumns(), expectedConstraint.getColumns()); + assertEquals(constraint.isEnforced(), expectedConstraint.isEnforced()); + assertEquals(constraint.isRely(), expectedConstraint.isRely()); + } + else { + assertEquals(constraint, expectedConstraint); + } + } + } + private static void rollbackIfEquals(TransactionDeleteInsertTestTag tag, TransactionDeleteInsertTestTag expectedTag) { if (expectedTag == tag) { diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveClientFileMetastore.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveClientFileMetastore.java index e67b39c214ce6..2b272ee0babde 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveClientFileMetastore.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveClientFileMetastore.java @@ -110,6 +110,12 @@ public void testTransactionDeleteInsert() // FileHiveMetastore has various incompatibilities } + @Override + public void testTableConstraints() + { + // FileHiveMetastore has no support for table constraints + } + @Test public void testPartitionNotReadable() { diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveClientInMemoryMetastore.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveClientInMemoryMetastore.java index f8ac01d461529..5d4563b513809 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveClientInMemoryMetastore.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveClientInMemoryMetastore.java @@ -42,4 +42,10 @@ public void testTransactionDeleteInsert() { // InMemoryHiveMetastore does not check whether partition exist in createPartition and dropPartition } + + @Override + public void testTableConstraints() + { + // InMemoryHiveMetastore has no support for table constraints + } } diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveClientInMemoryMetastoreWithFilterPushdown.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveClientInMemoryMetastoreWithFilterPushdown.java index e7199b343b0a7..90aac8bc109d2 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveClientInMemoryMetastoreWithFilterPushdown.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveClientInMemoryMetastoreWithFilterPushdown.java @@ -57,4 +57,10 @@ public void testTransactionDeleteInsert() { // InMemoryHiveMetastore does not check whether partition exist in createPartition and dropPartition } + + @Override + public void testTableConstraints() + { + // InMemoryHiveMetastore has no support for table constraints + } } diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveTableConstraints.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveTableConstraints.java new file mode 100644 index 0000000000000..c9a10af64829e --- /dev/null +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveTableConstraints.java @@ -0,0 +1,124 @@ +/* + * 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 com.facebook.presto.hive; + +import com.facebook.presto.cache.CacheConfig; +import com.facebook.presto.hive.authentication.NoHdfsAuthentication; +import com.facebook.presto.hive.metastore.HivePartitionMutator; +import com.facebook.presto.hive.metastore.SemiTransactionalHiveMetastore; +import com.facebook.presto.hive.metastore.thrift.BridgingHiveMetastore; +import com.facebook.presto.hive.metastore.thrift.HiveCluster; +import com.facebook.presto.hive.metastore.thrift.HiveMetastoreClient; +import com.facebook.presto.hive.metastore.thrift.MockHiveMetastoreClient; +import com.facebook.presto.hive.metastore.thrift.ThriftHiveMetastore; +import com.facebook.presto.hive.metastore.thrift.ThriftHiveMetastoreStats; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.constraints.PrimaryKeyConstraint; +import com.facebook.presto.spi.constraints.TableConstraint; +import com.facebook.presto.spi.constraints.UniqueConstraint; +import com.facebook.presto.testing.TestingConnectorSession; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.util.concurrent.ListeningExecutorService; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import java.util.List; +import java.util.Optional; + +import static com.facebook.airlift.concurrent.Threads.daemonThreadsNamed; +import static com.facebook.presto.hive.HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER; +import static com.facebook.presto.hive.HiveQueryRunner.METASTORE_CONTEXT; +import static com.facebook.presto.hive.metastore.thrift.MockHiveMetastoreClient.TEST_DATABASE; +import static com.facebook.presto.hive.metastore.thrift.MockHiveMetastoreClient.TEST_TABLE_WITH_CONSTRAINTS; +import static com.google.common.util.concurrent.MoreExecutors.listeningDecorator; +import static java.util.concurrent.Executors.newCachedThreadPool; +import static org.testng.Assert.assertEquals; + +@Test(singleThreaded = true) +public class TestHiveTableConstraints +{ + private MockHiveMetastoreClient mockClient; + private SemiTransactionalHiveMetastore metastore; + private ThriftHiveMetastoreStats stats; + + @BeforeMethod + public void setUp() + { + HiveClientConfig config = new HiveClientConfig(); + MetastoreClientConfig metastoreClientConfig = new MetastoreClientConfig(); + mockClient = new MockHiveMetastoreClient(); + MockHiveCluster mockHiveCluster = new MockHiveCluster(mockClient); + ListeningExecutorService executor = listeningDecorator(newCachedThreadPool(daemonThreadsNamed("test-%s"))); + PartitionMutator hivePartitionMutator = new HivePartitionMutator(); + HdfsConfiguration hdfsConfiguration = new HiveHdfsConfiguration(new HdfsConfigurationInitializer(config, metastoreClientConfig), ImmutableSet.of()); + HdfsEnvironment hdfsEnvironment = new HdfsEnvironment(hdfsConfiguration, metastoreClientConfig, new NoHdfsAuthentication()); + ThriftHiveMetastore thriftHiveMetastore = new ThriftHiveMetastore(mockHiveCluster, metastoreClientConfig, hdfsEnvironment); + metastore = new SemiTransactionalHiveMetastore( + hdfsEnvironment, + new BridgingHiveMetastore(thriftHiveMetastore, hivePartitionMutator), + executor, + false, + false, + true, + DEFAULT_COLUMN_CONVERTER_PROVIDER); + stats = thriftHiveMetastore.getStats(); + } + + @Test + public void testTableConstraints() + { + List> expectedConstraints = ImmutableList.of( + new PrimaryKeyConstraint<>("", ImmutableSet.of("c1"), true, true), + new UniqueConstraint<>("", ImmutableSet.of("c2"), true, true)); + + List> tableConstraints = metastore.getTableConstraints(METASTORE_CONTEXT, TEST_DATABASE, TEST_TABLE_WITH_CONSTRAINTS); + assertEquals(tableConstraints, expectedConstraints); + + ConnectorSession session = new TestingConnectorSession( + new HiveSessionProperties( + new HiveClientConfig().setMaxBucketsForGroupedExecution(100), + new OrcFileWriterConfig(), + new ParquetFileWriterConfig(), + new CacheConfig()) + .getSessionProperties()); + + metastore.dropTable(new HdfsContext(session, TEST_DATABASE, TEST_TABLE_WITH_CONSTRAINTS, "/some/path", false), TEST_DATABASE, TEST_TABLE_WITH_CONSTRAINTS); + tableConstraints = metastore.getTableConstraints(METASTORE_CONTEXT, TEST_DATABASE, TEST_TABLE_WITH_CONSTRAINTS); + assertEquals(tableConstraints, ImmutableList.of()); + } + + public static class MockHiveCluster + implements HiveCluster + { + private final MockHiveMetastoreClient client; + + public MockHiveCluster(MockHiveMetastoreClient client) + { + this.client = client; + } + + @Override + public HiveMetastoreClient createMetastoreClient(Optional token) + { + return client; + } + + public MockHiveMetastoreClient createPartitionVersionSupportedMetastoreClient() + { + return client; + } + } +} diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/metastore/TestCachingHiveMetastore.java b/presto-hive/src/test/java/com/facebook/presto/hive/metastore/TestCachingHiveMetastore.java index 38779f691d80d..aeaf4c8044a1a 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/metastore/TestCachingHiveMetastore.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/metastore/TestCachingHiveMetastore.java @@ -24,14 +24,19 @@ import com.facebook.presto.hive.metastore.thrift.MockHiveMetastoreClient; import com.facebook.presto.hive.metastore.thrift.ThriftHiveMetastore; import com.facebook.presto.hive.metastore.thrift.ThriftHiveMetastoreStats; +import com.facebook.presto.spi.constraints.PrimaryKeyConstraint; +import com.facebook.presto.spi.constraints.TableConstraint; +import com.facebook.presto.spi.constraints.UniqueConstraint; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.util.concurrent.ListeningExecutorService; import io.airlift.units.Duration; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.TimeUnit; @@ -51,6 +56,7 @@ import static com.facebook.presto.hive.metastore.thrift.MockHiveMetastoreClient.TEST_PARTITION_VALUES2; import static com.facebook.presto.hive.metastore.thrift.MockHiveMetastoreClient.TEST_ROLES; import static com.facebook.presto.hive.metastore.thrift.MockHiveMetastoreClient.TEST_TABLE; +import static com.facebook.presto.hive.metastore.thrift.MockHiveMetastoreClient.TEST_TABLE_WITH_CONSTRAINTS; import static com.google.common.util.concurrent.MoreExecutors.listeningDecorator; import static java.util.concurrent.Executors.newCachedThreadPool; import static java.util.function.UnaryOperator.identity; @@ -111,14 +117,14 @@ public void testGetAllDatabases() public void testGetAllTable() { assertEquals(mockClient.getAccessCount(), 0); - assertEquals(metastore.getAllTables(TEST_METASTORE_CONTEXT, TEST_DATABASE).get(), ImmutableList.of(TEST_TABLE)); + assertEquals(metastore.getAllTables(TEST_METASTORE_CONTEXT, TEST_DATABASE).get(), ImmutableList.of(TEST_TABLE, TEST_TABLE_WITH_CONSTRAINTS)); assertEquals(mockClient.getAccessCount(), 1); - assertEquals(metastore.getAllTables(TEST_METASTORE_CONTEXT, TEST_DATABASE).get(), ImmutableList.of(TEST_TABLE)); + assertEquals(metastore.getAllTables(TEST_METASTORE_CONTEXT, TEST_DATABASE).get(), ImmutableList.of(TEST_TABLE, TEST_TABLE_WITH_CONSTRAINTS)); assertEquals(mockClient.getAccessCount(), 1); metastore.flushCache(); - assertEquals(metastore.getAllTables(TEST_METASTORE_CONTEXT, TEST_DATABASE).get(), ImmutableList.of(TEST_TABLE)); + assertEquals(metastore.getAllTables(TEST_METASTORE_CONTEXT, TEST_DATABASE).get(), ImmutableList.of(TEST_TABLE, TEST_TABLE_WITH_CONSTRAINTS)); assertEquals(mockClient.getAccessCount(), 2); } @@ -447,6 +453,21 @@ public void testNoCacheExceptions() assertEquals(mockClient.getAccessCount(), 2); } + @Test + public void testTableConstraints() + { + assertEquals(mockClient.getAccessCount(), 0); + List> tableConstraints = metastore.getTableConstraints(TEST_METASTORE_CONTEXT, TEST_DATABASE, TEST_TABLE_WITH_CONSTRAINTS); + assertEquals(tableConstraints.get(0), new PrimaryKeyConstraint<>("", ImmutableSet.of("c1"), true, true)); + assertEquals(tableConstraints.get(1), new UniqueConstraint<>("", ImmutableSet.of("c2"), true, true)); + assertEquals(mockClient.getAccessCount(), 2); + metastore.getTableConstraints(TEST_METASTORE_CONTEXT, TEST_DATABASE, TEST_TABLE_WITH_CONSTRAINTS); + assertEquals(mockClient.getAccessCount(), 2); + metastore.flushCache(); + metastore.getTableConstraints(TEST_METASTORE_CONTEXT, TEST_DATABASE, TEST_TABLE_WITH_CONSTRAINTS); + assertEquals(mockClient.getAccessCount(), 4); + } + public static class MockHiveCluster implements HiveCluster { diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/metastore/glue/TestHiveClientGlueMetastore.java b/presto-hive/src/test/java/com/facebook/presto/hive/metastore/glue/TestHiveClientGlueMetastore.java index 47b339ae2b573..16adc5e423f79 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/metastore/glue/TestHiveClientGlueMetastore.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/metastore/glue/TestHiveClientGlueMetastore.java @@ -138,6 +138,12 @@ public void testUpdatePartitionColumnStatisticsEmptyOptionalFields() // column statistics are not supported by Glue } + @Override + public void testTableConstraints() + { + // GlueMetastore has no support for table constraints + } + @Override public void testStorePartitionWithStatistics() throws Exception diff --git a/presto-hive/src/test/sql/create-test-hive-3.sql b/presto-hive/src/test/sql/create-test-hive-3.sql index e69de29bb2d1d..6ceaab6f22a92 100644 --- a/presto-hive/src/test/sql/create-test-hive-3.sql +++ b/presto-hive/src/test/sql/create-test-hive-3.sql @@ -0,0 +1,47 @@ +CREATE TABLE test_constraints1( + c1 INT, + c2 INT, + c3 STRING, + c4 BIGINT, + c5 INT, + c6 BIGINT, + PRIMARY KEY(c1) DISABLE NOVALIDATE RELY, + CONSTRAINT UK1 UNIQUE (c2) DISABLE NOVALIDATE RELY +) + COMMENT 'Primary and Unique Key with single columns and RELY'; + +CREATE TABLE test_constraints2( + c1 INT, + c2 INT, + c3 STRING, + c4 BIGINT, + c5 INT, + c6 BIGINT, + PRIMARY KEY(c1,c2) DISABLE NOVALIDATE RELY, + CONSTRAINT UK2 UNIQUE (c3, c4) DISABLE NOVALIDATE RELY +) + COMMENT 'Primary and Unique Key with multiple columns and RELY'; + +CREATE TABLE test_constraints3( + c1 INT, + c2 INT, + c3 STRING, + c4 BIGINT, + c5 INT, + c6 BIGINT, + PRIMARY KEY(c1) DISABLE NOVALIDATE NORELY, + CONSTRAINT UK3 UNIQUE (c2) DISABLE NOVALIDATE NORELY +) + COMMENT 'Primary and Unique Key with single columns and NORELY'; + +CREATE TABLE test_constraints4( + c1 INT, + c2 INT, + c3 STRING, + c4 BIGINT, + c5 INT, + c6 BIGINT, + PRIMARY KEY(c1,c2) DISABLE NOVALIDATE NORELY, + CONSTRAINT UK4 UNIQUE (c3, c4) DISABLE NOVALIDATE NORELY +) + COMMENT 'Primary and Unique Key with multiple columns and NORELY'; diff --git a/presto-main/src/main/java/com/facebook/presto/FullConnectorSession.java b/presto-main/src/main/java/com/facebook/presto/FullConnectorSession.java index e08953c0d08c7..f98f476878dcf 100644 --- a/presto-main/src/main/java/com/facebook/presto/FullConnectorSession.java +++ b/presto-main/src/main/java/com/facebook/presto/FullConnectorSession.java @@ -28,6 +28,7 @@ import java.util.Optional; import java.util.Set; +import static com.facebook.presto.SystemSessionProperties.isExploitConstraints; import static com.facebook.presto.spi.StandardErrorCode.INVALID_SESSION_PROPERTY; import static com.google.common.base.MoreObjects.toStringHelper; import static java.lang.String.format; @@ -156,6 +157,12 @@ public Optional getSchema() return session.getSchema(); } + @Override + public boolean isReadConstraints() + { + return isExploitConstraints(session); + } + @Override public String toString() { diff --git a/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java b/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java index 2c3342f385d33..d4c85eb2f4da6 100644 --- a/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java +++ b/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java @@ -155,6 +155,7 @@ public final class SystemSessionProperties public static final String FILTER_AND_PROJECT_MIN_OUTPUT_PAGE_ROW_COUNT = "filter_and_project_min_output_page_row_count"; public static final String DISTRIBUTED_SORT = "distributed_sort"; public static final String USE_MARK_DISTINCT = "use_mark_distinct"; + public static final String EXPLOIT_CONSTRAINTS = "exploit_constraints"; public static final String PREFER_PARTIAL_AGGREGATION = "prefer_partial_aggregation"; public static final String PARTIAL_AGGREGATION_STRATEGY = "partial_aggregation_strategy"; public static final String PARTIAL_AGGREGATION_BYTE_REDUCTION_THRESHOLD = "partial_aggregation_byte_reduction_threshold"; @@ -825,6 +826,11 @@ public SystemSessionProperties( "Implement DISTINCT aggregations using MarkDistinct", featuresConfig.isUseMarkDistinct(), false), + booleanProperty( + EXPLOIT_CONSTRAINTS, + "Exploit table constraints.", + featuresConfig.isExploitConstraints(), + false), booleanProperty( PREFER_PARTIAL_AGGREGATION, "Prefer splitting aggregations into partial and final stages", @@ -1779,6 +1785,11 @@ public static boolean useMarkDistinct(Session session) return session.getSystemProperty(USE_MARK_DISTINCT, Boolean.class); } + public static boolean isExploitConstraints(Session session) + { + return session.getSystemProperty(EXPLOIT_CONSTRAINTS, Boolean.class); + } + public static PartialAggregationStrategy getPartialAggregationStrategy(Session session) { Boolean preferPartialAggregation = session.getSystemProperty(PREFER_PARTIAL_AGGREGATION, Boolean.class); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java index d421abba72d88..39b09b46b718b 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java @@ -150,6 +150,7 @@ public class FeaturesConfig private double memoryRevokingThreshold = 0.9; private boolean parseDecimalLiteralsAsDouble; private boolean useMarkDistinct = true; + private boolean exploitConstraints; private boolean preferPartialAggregation = true; private PartialAggregationStrategy partialAggregationStrategy = PartialAggregationStrategy.ALWAYS; private double partialAggregationByteReductionThreshold = 0.5; @@ -827,6 +828,18 @@ public FeaturesConfig setUseMarkDistinct(boolean value) return this; } + public boolean isExploitConstraints() + { + return exploitConstraints; + } + + @Config("optimizer.exploit-constraints") + public FeaturesConfig setExploitConstraints(boolean value) + { + this.exploitConstraints = value; + return this; + } + public boolean isPreferPartialAggregation() { return preferPartialAggregation; diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanOptimizers.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanOptimizers.java index 699f73fb5b2ed..a51bb1859d071 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanOptimizers.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanOptimizers.java @@ -24,6 +24,7 @@ import com.facebook.presto.sql.parser.SqlParser; import com.facebook.presto.sql.planner.iterative.IterativeOptimizer; import com.facebook.presto.sql.planner.iterative.Rule; +import com.facebook.presto.sql.planner.iterative.properties.LogicalPropertiesProviderImpl; import com.facebook.presto.sql.planner.iterative.rule.AddIntermediateAggregations; import com.facebook.presto.sql.planner.iterative.rule.CanonicalizeExpressions; import com.facebook.presto.sql.planner.iterative.rule.CreatePartialTopN; @@ -91,7 +92,13 @@ import com.facebook.presto.sql.planner.iterative.rule.PushTopNThroughUnion; import com.facebook.presto.sql.planner.iterative.rule.RemoveEmptyDelete; import com.facebook.presto.sql.planner.iterative.rule.RemoveFullSample; +import com.facebook.presto.sql.planner.iterative.rule.RemoveRedundantAggregateDistinct; +import com.facebook.presto.sql.planner.iterative.rule.RemoveRedundantDistinct; +import com.facebook.presto.sql.planner.iterative.rule.RemoveRedundantDistinctLimit; import com.facebook.presto.sql.planner.iterative.rule.RemoveRedundantIdentityProjections; +import com.facebook.presto.sql.planner.iterative.rule.RemoveRedundantLimit; +import com.facebook.presto.sql.planner.iterative.rule.RemoveRedundantSort; +import com.facebook.presto.sql.planner.iterative.rule.RemoveRedundantTopN; import com.facebook.presto.sql.planner.iterative.rule.RemoveTrivialFilters; import com.facebook.presto.sql.planner.iterative.rule.RemoveUnreferencedScalarApplyNodes; import com.facebook.presto.sql.planner.iterative.rule.RemoveUnreferencedScalarLateralNodes; @@ -140,6 +147,7 @@ import com.facebook.presto.sql.planner.optimizations.TransformQuantifiedComparisonApplyToLateralJoin; import com.facebook.presto.sql.planner.optimizations.UnaliasSymbolReferences; import com.facebook.presto.sql.planner.optimizations.WindowFilterPushDown; +import com.facebook.presto.sql.relational.FunctionResolution; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import org.weakref.jmx.MBeanExporter; @@ -149,6 +157,7 @@ import javax.inject.Inject; import java.util.List; +import java.util.Optional; import java.util.Set; import static com.facebook.presto.sql.planner.ConnectorPlanOptimizerManager.PlanPhase.LOGICAL; @@ -430,7 +439,14 @@ public PlanOptimizers( ruleStats, statsCalculator, estimatedExchangesCostCalculator, + Optional.of(new LogicalPropertiesProviderImpl(new FunctionResolution(metadata.getFunctionAndTypeManager()))), ImmutableSet.of( + new RemoveRedundantDistinct(), + new RemoveRedundantTopN(), + new RemoveRedundantSort(), + new RemoveRedundantLimit(), + new RemoveRedundantDistinctLimit(), + new RemoveRedundantAggregateDistinct(), new RemoveRedundantIdentityProjections(), new PushAggregationThroughOuterJoin(metadata.getFunctionAndTypeManager()))), inlineProjections, diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/RelationPlanner.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/RelationPlanner.java index 10992380764c0..8a2d75bc9aab8 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/RelationPlanner.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/RelationPlanner.java @@ -23,6 +23,7 @@ import com.facebook.presto.metadata.Metadata; import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.TableHandle; +import com.facebook.presto.spi.constraints.TableConstraint; import com.facebook.presto.spi.plan.AggregationNode; import com.facebook.presto.spi.plan.Assignments; import com.facebook.presto.spi.plan.ExceptNode; @@ -177,7 +178,9 @@ protected RelationPlan visitTable(Table node, Void context) } List outputVariables = outputVariablesBuilder.build(); - PlanNode root = new TableScanNode(getSourceLocation(node.getLocation()), idAllocator.getNextId(), handle, outputVariables, columns.build(), TupleDomain.all(), TupleDomain.all()); + List> tableConstraints = metadata.getTableMetadata(session, handle).getMetadata().getTableConstraints(); + PlanNode root = new TableScanNode(getSourceLocation(node.getLocation()), idAllocator.getNextId(), handle, outputVariables, columns.build(), tableConstraints, TupleDomain.all(), TupleDomain.all()); + return new RelationPlan(root, scope, outputVariables); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/GroupReference.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/GroupReference.java index 4d6a1daa5d92a..7327d04267471 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/GroupReference.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/GroupReference.java @@ -14,6 +14,7 @@ package com.facebook.presto.sql.planner.iterative; import com.facebook.presto.spi.SourceLocation; +import com.facebook.presto.spi.plan.LogicalProperties; import com.facebook.presto.spi.plan.PlanNode; import com.facebook.presto.spi.plan.PlanNodeId; import com.facebook.presto.spi.relation.VariableReferenceExpression; @@ -29,12 +30,14 @@ public class GroupReference { private final int groupId; private final List outputs; + private final Optional logicalProperties; - public GroupReference(Optional sourceLocation, PlanNodeId id, int groupId, List outputs) + public GroupReference(Optional sourceLocation, PlanNodeId id, int groupId, List outputs, Optional logicalProperties) { super(sourceLocation, id); this.groupId = groupId; this.outputs = ImmutableList.copyOf(outputs); + this.logicalProperties = logicalProperties; } public int getGroupId() @@ -65,4 +68,9 @@ public PlanNode replaceChildren(List newChildren) { throw new UnsupportedOperationException(); } + + public Optional getLogicalProperties() + { + return logicalProperties; + } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/IterativeOptimizer.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/IterativeOptimizer.java index d4dd17cdc780d..1b8d0a465cdc5 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/IterativeOptimizer.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/IterativeOptimizer.java @@ -25,6 +25,7 @@ import com.facebook.presto.matching.Matcher; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.WarningCollector; +import com.facebook.presto.spi.plan.LogicalPropertiesProvider; import com.facebook.presto.spi.plan.PlanNode; import com.facebook.presto.spi.plan.PlanNodeIdAllocator; import com.facebook.presto.sql.planner.PlanVariableAllocator; @@ -56,13 +57,24 @@ public class IterativeOptimizer private final CostCalculator costCalculator; private final List legacyRules; private final RuleIndex ruleIndex; + private final Optional logicalPropertiesProvider; public IterativeOptimizer(RuleStatsRecorder stats, StatsCalculator statsCalculator, CostCalculator costCalculator, Set> rules) { - this(stats, statsCalculator, costCalculator, ImmutableList.of(), rules); + this(stats, statsCalculator, costCalculator, ImmutableList.of(), Optional.empty(), rules); + } + + public IterativeOptimizer(RuleStatsRecorder stats, StatsCalculator statsCalculator, CostCalculator costCalculator, Optional logicalPropertiesProvider, Set> rules) + { + this(stats, statsCalculator, costCalculator, ImmutableList.of(), logicalPropertiesProvider, rules); } public IterativeOptimizer(RuleStatsRecorder stats, StatsCalculator statsCalculator, CostCalculator costCalculator, List legacyRules, Set> newRules) + { + this(stats, statsCalculator, costCalculator, legacyRules, Optional.empty(), newRules); + } + + public IterativeOptimizer(RuleStatsRecorder stats, StatsCalculator statsCalculator, CostCalculator costCalculator, List legacyRules, Optional logicalPropertiesProvider, Set> newRules) { this.stats = requireNonNull(stats, "stats is null"); this.statsCalculator = requireNonNull(statsCalculator, "statsCalculator is null"); @@ -71,6 +83,7 @@ public IterativeOptimizer(RuleStatsRecorder stats, StatsCalculator statsCalculat this.ruleIndex = RuleIndex.builder() .register(newRules) .build(); + this.logicalPropertiesProvider = requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider is null"); stats.registerAll(newRules); } @@ -87,7 +100,14 @@ public PlanNode optimize(PlanNode plan, Session session, TypeProvider types, Pla return plan; } - Memo memo = new Memo(idAllocator, plan); + Memo memo; + if (SystemSessionProperties.isExploitConstraints(session)) { + memo = new Memo(idAllocator, plan, logicalPropertiesProvider); + } + else { + memo = new Memo(idAllocator, plan, Optional.empty()); + } + Lookup lookup = Lookup.from(planNode -> Stream.of(memo.resolve(planNode))); Matcher matcher = new PlanNodeMatcher(lookup); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/Memo.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/Memo.java index 679e0c497fde2..65a5bfbfdfffe 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/Memo.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/Memo.java @@ -15,6 +15,8 @@ import com.facebook.presto.cost.PlanCostEstimate; import com.facebook.presto.cost.PlanNodeStatsEstimate; +import com.facebook.presto.spi.plan.LogicalProperties; +import com.facebook.presto.spi.plan.LogicalPropertiesProvider; import com.facebook.presto.spi.plan.PlanNode; import com.facebook.presto.spi.plan.PlanNodeIdAllocator; import com.google.common.collect.HashMultiset; @@ -69,12 +71,19 @@ public class Memo private final int rootGroup; private final Map groups = new HashMap<>(); + private final Optional logicalPropertiesProvider; private int nextGroupId = ROOT_GROUP_REF + 1; public Memo(PlanNodeIdAllocator idAllocator, PlanNode plan) + { + this(idAllocator, plan, Optional.empty()); + } + + public Memo(PlanNodeIdAllocator idAllocator, PlanNode plan, Optional logicalPropertiesProvider) { this.idAllocator = idAllocator; + this.logicalPropertiesProvider = logicalPropertiesProvider; rootGroup = insertRecursive(plan); groups.get(rootGroup).incomingReferences.add(ROOT_GROUP_REF); } @@ -90,6 +99,12 @@ private Group getGroup(int group) return groups.get(group); } + public Optional getLogicalProperties(int group) + { + checkArgument(groups.containsKey(group), "Invalid group: %s", group); + return groups.get(group).logicalProperties; + } + public PlanNode getNode(int group) { return getGroup(group).membership; @@ -129,6 +144,14 @@ public PlanNode replace(int group, PlanNode node, String reason) incrementReferenceCounts(node, group); getGroup(group).membership = node; + + if (logicalPropertiesProvider.isPresent()) { + // for now, we replace existing group logical properties with those computed for the new node + // as we cannot ensure equivalence for all plans in a group until we support functional dependencies + // once we can ensure equivalence we can simply reuse the previously computed properties for all plans in the group + LogicalProperties newLogicalProperties = node.computeLogicalProperties(logicalPropertiesProvider.get()); + getGroup(group).logicalProperties = Optional.of(newLogicalProperties); + } decrementReferenceCounts(old, group); evictStatisticsAndCost(group); @@ -212,11 +235,15 @@ private PlanNode insertChildrenAndRewrite(PlanNode node) { return node.replaceChildren( node.getSources().stream() - .map(child -> new GroupReference( - node.getSourceLocation(), - idAllocator.getNextId(), - insertRecursive(child), - child.getOutputVariables())) + .map(child -> { + int childId = insertRecursive(child); + return new GroupReference( + node.getSourceLocation(), + idAllocator.getNextId(), + childId, + child.getOutputVariables(), + groups.get(childId).logicalProperties); + }) .collect(Collectors.toList())); } @@ -229,7 +256,7 @@ private int insertRecursive(PlanNode node) int group = nextGroupId(); PlanNode rewritten = insertChildrenAndRewrite(node); - groups.put(group, Group.withMember(rewritten)); + groups.put(group, new Group(rewritten, logicalPropertiesProvider.map(rewritten::computeLogicalProperties))); incrementReferenceCounts(rewritten, group); return group; @@ -247,21 +274,18 @@ public int getGroupCount() private static final class Group { - static Group withMember(PlanNode member) - { - return new Group(member); - } - - private PlanNode membership; private final Multiset incomingReferences = HashMultiset.create(); + private PlanNode membership; + private Optional logicalProperties; @Nullable private PlanNodeStatsEstimate stats; @Nullable private PlanCostEstimate cost; - private Group(PlanNode member) + private Group(PlanNode member, Optional logicalProperties) { this.membership = requireNonNull(member, "member is null"); + this.logicalProperties = logicalProperties; } } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/EquivalenceClassProperty.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/EquivalenceClassProperty.java new file mode 100644 index 0000000000000..c683738fb22ed --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/EquivalenceClassProperty.java @@ -0,0 +1,311 @@ +/* + * 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 com.facebook.presto.sql.planner.iterative.properties; + +import com.facebook.presto.spi.relation.CallExpression; +import com.facebook.presto.spi.relation.ConstantExpression; +import com.facebook.presto.spi.relation.RowExpression; +import com.facebook.presto.spi.relation.VariableReferenceExpression; +import com.facebook.presto.sql.relational.FunctionResolution; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static com.facebook.presto.expressions.LogicalRowExpressions.extractConjuncts; +import static com.google.common.base.MoreObjects.toStringHelper; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static java.util.Objects.requireNonNull; + +/** + * Represents classes of equivalent variable and constants references that + * hold for a final or intermediate result set produced by a PlanNode. + * Variable and constant references are made equivalent via predicate application. + *

+ * Each equivalence class is represented by a head. The head is carefully chosen + * to be a the member with the strongest binding (i.e. a constant). + *

+ * Note that the equivalence class property does not store trivial equivalence + * classes with only one member. All variable or constant references are + * considered to be in their own virtual equivalence class until combined + * into stored larger classes by predicate application. + */ +public class EquivalenceClassProperty +{ + private final Map equivalenceClassHeads = new HashMap<>(); + private final Map> equivalenceClasses = new HashMap<>(); + private final FunctionResolution functionResolution; + + private boolean updated; + + public EquivalenceClassProperty(FunctionResolution functionResolution) + { + this.functionResolution = functionResolution; + } + + /** + * Determines if one equivalence class property is more general than another. + * An equivalence class property is more general than another if it includes all equivalences of the other. + * + * @param otherEquivalenceClassProperty + * @return True if this equivalenceClassProperty is more general than otherEquivalenceClassProperty or False otherwise. + */ + public boolean isMoreGeneralThan(EquivalenceClassProperty otherEquivalenceClassProperty) + { + requireNonNull(otherEquivalenceClassProperty, "otherEquivalenceClassProperty is null"); + if (equivalenceClasses.isEmpty() && otherEquivalenceClassProperty.equivalenceClasses.isEmpty()) { + return true; + } + if (equivalenceClasses.isEmpty() || otherEquivalenceClassProperty.equivalenceClasses.isEmpty()) { + return false; + } + + ImmutableList> eqClassSets = equivalenceClasses.entrySet() + .stream() + .map(e1 -> new ImmutableSet.Builder() + .add(e1.getKey()) + .addAll(e1.getValue()) + .build()) + .collect(toImmutableList()); + + //every equivalence class of other is a subset of some equivalence class of the first + return otherEquivalenceClassProperty.equivalenceClasses.entrySet() + .stream() + .allMatch(e -> { + final Set otherEqClass = new HashSet<>(); + otherEqClass.add(e.getKey()); + otherEqClass.addAll(e.getValue()); + return eqClassSets.stream().anyMatch(eqClassSet -> eqClassSet.containsAll(otherEqClass)); + }); + } + + /** + * Returns the head of the equivalence class of the provided variable or constant reference. + * + * @param expression + * @return The head of the equivalence class of the provided variable or constant reference. + */ + public RowExpression getEquivalenceClassHead(RowExpression expression) + { + requireNonNull(expression, "expression is null"); + checkArgument((expression instanceof VariableReferenceExpression || expression instanceof ConstantExpression), + "Row expression is of type " + expression.getClass().getSimpleName() + ", must be a VariableReferenceExpression or a ConstantExpression."); + //all variables start out by default in their own virtual singleton class + return equivalenceClassHeads.getOrDefault(expression, expression); + } + + /** + * Returns the equivalence classes members for the given equivalence class head. + *

+ * Note that the provided head could be that of a new equivalence class in which case + * an empty member list is returned. + * + * @param head + * @return The equivalence class members for the given equivalence class head. + */ + public List getEquivalenceClasses(RowExpression head) + { + requireNonNull(head, "head is null"); + checkArgument((head instanceof VariableReferenceExpression || head instanceof ConstantExpression), + "Row expression is of type " + head.getClass().getSimpleName() + ", must be a VariableReferenceExpression or a ConstantExpression."); + + return equivalenceClasses.getOrDefault(head, new ArrayList<>()); + } + + /** + * Updates this equivalence class property with the equivalences of another equivalence class property. + * + * @param equivalenceClassProperty + */ + public void update(EquivalenceClassProperty equivalenceClassProperty) + { + requireNonNull(equivalenceClassProperty, "equivalenceClassProperty is null"); + + equivalenceClassProperty.equivalenceClasses.entrySet().forEach(eqClass -> { + final RowExpression head = eqClass.getKey(); + List members = eqClass.getValue(); + members.forEach(member -> updateInternal(head, member)); + }); + } + + /** + * Updates this equivalence class property with "variable reference = variable reference" or + * "variable reference = constant reference" conjuncts applied by the provided predicate. + * Returns true if any equivalence class heads changed, and false otherwise. + * + * @param predicate + * @return + */ + public boolean update(RowExpression predicate) + { + requireNonNull(predicate, "predicate is null"); + updated = false; + //TODO tunnel through CAST functions? + extractConjuncts(predicate).stream() + .filter(CallExpression.class::isInstance) + .map(CallExpression.class::cast) + .filter(e -> isVariableEqualVariableOrConstant(e)) + .forEach(e -> updateInternal(e.getArguments().get(0), e.getArguments().get(1))); + return updated; + } + + private boolean isVariableEqualVariableOrConstant(RowExpression expression) + { + if (expression instanceof CallExpression + && functionResolution.isEqualFunction(((CallExpression) expression).getFunctionHandle()) + && ((CallExpression) expression).getArguments().size() == 2) { + RowExpression e1 = ((CallExpression) expression).getArguments().get(0); + RowExpression e2 = ((CallExpression) expression).getArguments().get(1); + + if ((e1 instanceof VariableReferenceExpression && (e2 instanceof VariableReferenceExpression || e2 instanceof ConstantExpression)) + || ((e2 instanceof VariableReferenceExpression) && (e1 instanceof VariableReferenceExpression || e1 instanceof ConstantExpression))) { + return true; + } + } + return false; + } + + public void update(RowExpression firstExpression, RowExpression secondExpression) + { + updateInternal(firstExpression, secondExpression); + } + + /** + * Updates this equivalence class property with pairs of variable or column references deemed + * equivalent via the application of predicates. + * Side effect is that it sets updated instance variable true if any equivalence class heads changed. + * This can be used to optimize methods that maintain alignment with equivalence classes. + * + * @param firstExpression + * @param secondExpression + */ + private void updateInternal(RowExpression firstExpression, RowExpression secondExpression) + { + RowExpression head1 = getEquivalenceClassHead(firstExpression); + RowExpression head2 = getEquivalenceClassHead(secondExpression); + + //already in same equivalence class, nothing to do + //note that we do not check head1.equal(head2) so that two different variable reference objects + //referencing the same reference are both added to the equivalence class + if (head1 == head2) { + return; + } + + updated = true; + List head1Class = getEquivalenceClasses(head1); + List head2Class = getEquivalenceClasses(head2); + + //pick new head and merge other class into head class + RowExpression newHead = pickNewHead(head1, head2); + if (newHead == head1) { + combineClasses(head1, head1Class, head2, head2Class); + } + else { + combineClasses(head2, head2Class, head1, head1Class); + } + } + + private RowExpression pickNewHead(RowExpression head1, RowExpression head2) + { + //always use constant as the head + if (head1 instanceof ConstantExpression) { + return head1; + } + else if (head2 instanceof ConstantExpression) { + return head2; + } + else { + return head1; + } + } + + //combine an equivalence class with head class + private void combineClasses(RowExpression head, List headClass, RowExpression headOfOtherEqClass, List otherEqClass) + { + //merge other eq class into head class + headClass.addAll(otherEqClass); + headClass.add(headOfOtherEqClass); + //update the head of the other class members + equivalenceClassHeads.put(headOfOtherEqClass, head); + for (RowExpression expression : otherEqClass) { + equivalenceClassHeads.put(expression, head); + } + equivalenceClasses.putIfAbsent(head, headClass); + equivalenceClasses.remove(headOfOtherEqClass); + } + + /** + * Returns a projected version of this equivalence class property. + * Variables in each class are mapped to output variables in the context beyond the project operation. + * It is possible that this operation projects all members of a particular class. + * + * @param inverseVariableMappings + * @return A projected version of this equivalence class property. + */ + public EquivalenceClassProperty project(Map inverseVariableMappings) + { + EquivalenceClassProperty projectedEquivalenceClassProperty = new EquivalenceClassProperty(functionResolution); + for (Map.Entry> entry : equivalenceClasses.entrySet()) { + //first project the members of the current class + List projectedMembers = new ArrayList<>(); + for (RowExpression member : entry.getValue()) { + if (inverseVariableMappings.containsKey(member)) { + RowExpression projectedMember = inverseVariableMappings.get(member); + if (!projectedMembers.contains(projectedMember)) { + projectedMembers.add(projectedMember); + } + } + } + //boundary cases.... + // head projects but no members project -> trivial class, do not store + // head does not project and one member projects -> trivial class, do not store + // head does not project and more than one member projects -> pick a new head and update equivalence class + if (!projectedMembers.isEmpty()) { + RowExpression currentHead = entry.getKey(); + RowExpression projectedHead = currentHead; + if ((currentHead instanceof VariableReferenceExpression)) { + if (inverseVariableMappings.containsKey(currentHead)) { + //head is not projected + projectedHead = inverseVariableMappings.get(currentHead); + projectedMembers.remove(projectedHead); + } + else { + //pick the first projected member as the new head + projectedHead = projectedMembers.get(0); + projectedMembers.remove(0); + } + } + + RowExpression finalProjectedHead = projectedHead; + projectedMembers.forEach(m -> projectedEquivalenceClassProperty.update(finalProjectedHead, m)); + } + } + return projectedEquivalenceClassProperty; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("EquivalenceClassHeads", String.join(",", equivalenceClassHeads.entrySet().stream().map(e -> e.getKey().toString() + ":" + e.getValue().toString()).collect(toImmutableList()))) + .add("EquivalenceClasses", String.join(",", equivalenceClasses.entrySet().stream().map(e -> e.getKey().toString() + ":" + e.getValue().toString()).collect(toImmutableList()))) + .toString(); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/Key.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/Key.java new file mode 100644 index 0000000000000..847e103541831 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/Key.java @@ -0,0 +1,151 @@ +/* + * 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 com.facebook.presto.sql.planner.iterative.properties; + +import com.facebook.presto.spi.relation.ConstantExpression; +import com.facebook.presto.spi.relation.RowExpression; +import com.facebook.presto.spi.relation.VariableReferenceExpression; + +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +/** + * Represents a primary or unique key constraint that holds for a final or + * intermediate result set produced by a PlanNode. + * It can also be used to represent a key requirement that must be + * satisfied by a PlanNode (e.g. distinct requirement) + */ +public class Key +{ + private final Set variables; + + /** + * A set of variable references that satisfy a primary or unique key constraint. + * + * @param variables + */ + public Key(Set variables) + { + requireNonNull(variables, "Variables is null."); + checkArgument(!variables.isEmpty(), "Variables is empty"); + this.variables = variables; + } + + /** + * Determines if a provided key requirement is satisfied by this key. + * This is true if the variables in this key are a subset of the variables in the key requirement. + * Note the this operation should be called only after using the normalize method to render + * the key and key requirement into their canonical forms using equivalence classes. + * + * @param keyRequirement + * @return True if this key satisfies the key requirement and False otherwise. + */ + public boolean keySatisifiesRequirement(Key keyRequirement) + { + requireNonNull(keyRequirement, "Key requirement must be provided."); + + //ideally this would be a simple subset operation but the "canonicalize" operation in UnliasSymbols inexplicably + //clones VariableReferenceExpression's so two references to the same outputs might be made via different objects + return variables.stream().allMatch(vk -> keyRequirement.variables.stream().anyMatch(vr -> vk.equals(vr))); + } + + /** + * Returns a canonical version of this key wherein duplicate or constant variables are removed + * and any remaining variables are replaced with their equivalence class heads. + * Note that if all key variables are bound to constants an empty result is + * returned, signaling that at most a single record is in the result set constrained + * by this key. + * + * @param equivalenceClassProperty + * @return A normalized version of this key or empty if all variables are bound to constants. + */ + public Optional normalize(EquivalenceClassProperty equivalenceClassProperty) + { + requireNonNull(equivalenceClassProperty, "Equivalence class property must be provided."); + Set unBoundVariables = new HashSet<>(); + variables.stream().forEach(v -> { + RowExpression eqHead = equivalenceClassProperty.getEquivalenceClassHead(v); + if (!(eqHead instanceof ConstantExpression)) { + unBoundVariables.add((VariableReferenceExpression) eqHead); + } + }); + if (unBoundVariables.isEmpty()) { + return Optional.empty(); + } + else { + return Optional.of(new Key(unBoundVariables)); + } + } + + /** + * Returns a projected version of this key. + * Variables in the key are mapped to output variables in the context beyond the project operation. + * If a key attribute does not have an assignment in the new attribute context, it is mapped to the assignment of + * an equivalent attribute whenever possible. For example, assume A is a key attribute and there is no new assignment + * for A. Assume further that A and B are in the same equivalence class and there is an assignment from B to B’. + * Consequently, A can be assigned to B' rather than get projected. If any of the variables are not mapped then an + * empty result is returned signaling that the key is effectively uninteresting beyond the project operation and hence is not propagated. + * + * @param inverseVariableMappings + * @return A projected version of this key or empty if any variables are not propagated. + */ + public Optional project(LogicalPropertiesImpl.InverseVariableMappingsWithEquivalence inverseVariableMappings) + { + requireNonNull(inverseVariableMappings, "Inverse variable mappings must be provided."); + Set mappedVariables = new HashSet<>(); + Optional mappedVariable; + for (VariableReferenceExpression v : variables) { + mappedVariable = inverseVariableMappings.get(v); + if (mappedVariable.isPresent()) { + mappedVariables.add(mappedVariable.get()); + continue; + } + else { + return Optional.empty(); + } + } + return Optional.of(new Key(mappedVariables)); + } + + /** + * Returns a version of this key concatenated with the provided key. + * A concatenated key results from a join operation where concatenated keys of the left and + * right join inputs form unique constraints on the join result. + * + * @param toConcatKey + * @return a version of this key concatenated with the provided key. + */ + public Key concat(Key toConcatKey) + { + requireNonNull(toConcatKey, "Key must be provided."); + Set concatenatedVariables = new HashSet(); + concatenatedVariables.addAll(this.variables); + concatenatedVariables.addAll(toConcatKey.variables); + return new Key(concatenatedVariables); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("variables", String.join(",", variables.stream().map(VariableReferenceExpression::toString).collect(Collectors.toList()))) + .toString(); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/KeyProperty.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/KeyProperty.java new file mode 100644 index 0000000000000..c09323816b756 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/KeyProperty.java @@ -0,0 +1,216 @@ +/* + * 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 com.facebook.presto.sql.planner.iterative.properties; + +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +/** + * Represents a collection of primary or unique key constraints that hold for a final or + * intermediate result set produced by a PlanNode. + */ +public class KeyProperty +{ + private final Set keys; + + public KeyProperty() + { + this.keys = new HashSet<>(); + } + + public KeyProperty(Set keys) + { + this.keys = keys; + } + + /** + * Determines if one key property is more general than another. + * A key property is more general than another if it can satisfy any key requirement the other can satisfy. + * + * @param otherKeyProperty + * @return True keyProperty is more general than otherKeyProperty or False otherwise. + */ + public boolean moreGeneral(KeyProperty otherKeyProperty) + { + requireNonNull(otherKeyProperty, "otherKeyProperty is null"); + return ((keys.isEmpty() && otherKeyProperty.keys.isEmpty()) || + (otherKeyProperty.keys.stream().allMatch(k -> satisfiesKeyRequirement(k)))); + } + + /** + * Determines if this key property satisfies a key requirement. + * This is true if any of the keys in the collection satisfies the key requirement. + * + * @param keyRequirement + * @return True if keyRequirement is satisfied by this key property or False otherwise. + */ + public boolean satisfiesKeyRequirement(Key keyRequirement) + { + requireNonNull(keyRequirement, "keyRequirement is null"); + return keys.stream().anyMatch(k -> k.keySatisifiesRequirement(keyRequirement)); + } + + /** + * Adds a set of keys to this key property. + * + * @param keys + */ + public void addKeys(Set keys) + { + requireNonNull(keys, "keys is null"); + keys.stream().forEach(k -> addKey(k)); + } + + /** + * Adds the keys from the provided key property to this key property. + * + * @param keyProperty + */ + public void addKeys(KeyProperty keyProperty) + { + requireNonNull(keyProperty, "keyProperty is null"); + addKeys(keyProperty.keys); + } + + /** + * Adds a new key to this key property. + * + * @param key + */ + public void addKey(Key key) + { + requireNonNull(key, "key is null"); + addNonRedundantKey(key); + } + + /** + * Adds a key to this key property while enforcing the constraint that no + * key is redundant with respect to another. + * E.g. if {orderkey} was an existing key then the key {orderkey, orderpriority} + * would represent a redundant key. The inverse is true, an existing key + * can be removed by a new key it if is redundant with respect to the new key. + * + * @param newKey + */ + private void addNonRedundantKey(Key newKey) + { + requireNonNull(newKey, "newKey is null"); + Set removedKeys = new HashSet<>(); + for (Key key : keys) { + //if the new key >= key don't add it + if (key.keySatisifiesRequirement(newKey)) { + return; + } + + //if the new key <= key1 remove existing key. note that if this is true the new key will be added as it + //cannot be a superset of another key2 otherwise key2 <= key1 which violates the key property invariant + if (newKey.keySatisifiesRequirement(key)) { + removedKeys.add(key); + } + } + //new key not >= existing key + keys.add(newKey); + keys.removeAll(removedKeys); + } + + /** + * Reduces key property to a concise cannonical form wherein each individual key is + * reduced to a canonical form by removing redundant variables and replacing any remaining variables + * with their equivalence class heads. Moreover, no keys in the normalized key + * property are redundant with respect to the others. + * Note that if any key is fully bound to constants an empty result is + * returned, signaling that at most a single record is in the result set constrained + * by this key property. + * + * @param equivalenceClassProperty + * @return A normalized version of this key property or empty if any key is fully bound to constants. + */ + public Optional normalize(EquivalenceClassProperty equivalenceClassProperty) + { + requireNonNull(equivalenceClassProperty, "equivalenceClassProperty is null"); + KeyProperty result = new KeyProperty(); + for (Key key : this.keys) { + Optional normalizedKey = key.normalize(equivalenceClassProperty); + if (!normalizedKey.isPresent()) { + return Optional.empty(); + } + else { + result.addKey(normalizedKey.get()); + } + } + return Optional.of(result); + } + + /** + * Returns a projected version of this key property. + * Variables in each key are mapped to output variables in the context beyond the project operation. + * It is possible that this operation projects all keys from the key property. + * + * @param inverseVariableMappings + * @return A projected version of this key property. + */ + public KeyProperty project(LogicalPropertiesImpl.InverseVariableMappingsWithEquivalence inverseVariableMappings) + { + requireNonNull(inverseVariableMappings, "inverseVariableMappings is null"); + KeyProperty result = new KeyProperty(); + keys.stream().forEach(key -> { + Optional projectedKey = key.project(inverseVariableMappings); + if (projectedKey.isPresent()) { + result.addKey(projectedKey.get()); + } + }); + return result; + } + + /** + * Empties all keys from the key property. + */ + public void empty() + { + keys.clear(); + } + + /** + * Returns a version of this key property wherein each key is concatenated with all keys in the provided key property + * A concatenated key property results from a join operation where concatenated keys of the left and + * right join inputs form unique constraints on the join result. + * + * @param toConcatKeyProp + * @return a version of this key concatenated with the provided key. + */ + public KeyProperty concat(KeyProperty toConcatKeyProp) + { + requireNonNull(toConcatKeyProp, "toConcatKeyProp is null"); + KeyProperty result = new KeyProperty(); + for (Key thisKey : this.keys) { + for (Key toConcatKey : toConcatKeyProp.keys) { + result.addKey(thisKey.concat(toConcatKey)); + } + } + return result; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("keys", String.join(",", keys.stream().map(Key::toString).collect(Collectors.toList()))) + .toString(); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/LogicalPropertiesImpl.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/LogicalPropertiesImpl.java new file mode 100644 index 0000000000000..0948cafd41c48 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/LogicalPropertiesImpl.java @@ -0,0 +1,644 @@ +/* + * 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 com.facebook.presto.sql.planner.iterative.properties; + +import com.facebook.presto.spi.plan.Assignments; +import com.facebook.presto.spi.plan.LogicalProperties; +import com.facebook.presto.spi.relation.RowExpression; +import com.facebook.presto.spi.relation.VariableReferenceExpression; +import com.facebook.presto.sql.planner.plan.AssignmentUtils; +import com.facebook.presto.sql.planner.plan.JoinNode; +import com.facebook.presto.sql.relational.FunctionResolution; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +import static com.facebook.presto.sql.planner.plan.JoinNode.Type.FULL; +import static com.facebook.presto.sql.planner.plan.JoinNode.Type.INNER; +import static com.facebook.presto.sql.planner.plan.JoinNode.Type.LEFT; +import static com.facebook.presto.sql.planner.plan.JoinNode.Type.RIGHT; +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +/** + * Provides an implementation of interface LogicalProperties along with a set + * of builders that various PlanNode's can use to compute their logical properties. + *

+ * The logical properties of a PlanNode represent properties that hold for the final + * or intermediate result produced by the PlanNode and are a function of the logical properties + * of the PlanNode's source(s) and the operation performed by the PlanNode. + * For example, and AggregationNode with a single grouping key + * would add a unique key to the properties of its input source. + *

+ * Note that for this implementation to work effectively it must sit behind the TranslateExpressions + * optimizer as it does not currently deal with original expressions. The TranslateExpressions + * functionality should ultimately be moved earlier into query compilation as opposed to + * extending this implementation with support for original expressions. + */ +public class LogicalPropertiesImpl + implements LogicalProperties +{ + private final MaxCardProperty maxCardProperty; + private final KeyProperty keyProperty; + private final EquivalenceClassProperty equivalenceClassProperty; + + public LogicalPropertiesImpl(EquivalenceClassProperty equivalenceClassProperty, MaxCardProperty maxCardProperty, KeyProperty keyProperty) + { + this.equivalenceClassProperty = requireNonNull(equivalenceClassProperty, "equivalenceClassProperty is null"); + this.maxCardProperty = requireNonNull(maxCardProperty, "maxCardProperty is null"); + this.keyProperty = requireNonNull(keyProperty, "keyProperty is null"); + } + + private static void checkArgument(boolean condition, String message) + { + if (!condition) { + throw new IllegalArgumentException(message); + } + } + + /** + * Determines if one set of logical properties is more general than another set. + * A set of logical properties is more general than another set if they can satisfy + * any requirement the other can satisfy. See the corresponding moreGeneral method + * for each of the individual properties to get more detail on the overall semantics. + * + * @param otherLogicalProperties + * @return True if this logicalproperties is more general than otherLogicalProperties or False otherwise. + */ + private boolean isMoreGeneralThan(LogicalPropertiesImpl otherLogicalProperties) + { + requireNonNull(otherLogicalProperties, "otherLogicalProperties is null"); + return (this.maxCardProperty.moreGeneral(otherLogicalProperties.maxCardProperty) && + this.keyProperty.moreGeneral(otherLogicalProperties.keyProperty) && + this.equivalenceClassProperty.isMoreGeneralThan(otherLogicalProperties.equivalenceClassProperty)); + } + + /** + * Determines if two sets of logical properties are equivalent. + * Two sets of logical properties are equivalent if each is more general than the other. + * + * @param otherLogicalProperties + * @return True if this and otherLogicalProperties are equivalent or False otherwise. + */ + public boolean equals(LogicalPropertiesImpl otherLogicalProperties) + { + requireNonNull(otherLogicalProperties, "otherLogicalProperties is null"); + return ((this.isMoreGeneralThan(otherLogicalProperties)) && otherLogicalProperties.isMoreGeneralThan(this)); + } + + /** + * Produces the inverse mapping of the provided assignments. + * The inverse mapping is used to propagate individual properties across a project operation + * by rewriting the property's variable references to those of the + * output of the project operation as per the provided assignments. + */ + private static Map inverseVariableAssignments(Assignments assignments) + { + //TODO perhaps put this in AssignmentsUtils or ProjectUtils + requireNonNull(assignments, "assignments is null"); + Map inverseVariableAssignments = new HashMap<>(); + for (Map.Entry e : assignments.entrySet()) { + if (e.getValue() instanceof VariableReferenceExpression) { + inverseVariableAssignments.put((VariableReferenceExpression) e.getValue(), e.getKey()); + } + } + return inverseVariableAssignments; + } + + /** + * Encapsulates normalization of the key property in alignment with equivalence class property, + * and possible setting of max card property if a one record condition is detected. + * The key property is modified. Maxcard will be modified if a one record condition is detected. + */ + private static void normalizeKeyPropertyAndSetMaxCard(KeyProperty keyProperty, MaxCardProperty maxCardProperty, EquivalenceClassProperty equivalenceClassProperty) + { + if (maxCardProperty.isAtMostOne()) { + keyProperty.empty(); + return; + } + Optional normalizedKeyProperty = keyProperty.normalize(equivalenceClassProperty); + keyProperty.empty(); //add in normalized keys or set maxcard + if (normalizedKeyProperty.isPresent()) { + keyProperty.addKeys(normalizedKeyProperty.get()); + } + else { + maxCardProperty.update(1); + } + } + + @Override + public boolean isDistinct(Set keyVars) + { + requireNonNull(keyVars, "keyVars is null"); + checkArgument(!keyVars.isEmpty(), "keyVars is empty"); + return this.keyRequirementSatisfied(new Key(keyVars)); + } + + @Override + public boolean isAtMostSingleRow() + { + return this.isAtMost(1); + } + + @Override + public boolean isAtMost(long n) + { + return maxCardProperty.isAtMost(n); + } + + private boolean keyRequirementSatisfied(Key keyRequirement) + { + requireNonNull(keyRequirement, "keyRequirement is null"); + if (maxCardProperty.isAtMostOne()) { + return true; + } + Optional normalizedKeyRequirement = keyRequirement.normalize(equivalenceClassProperty); + if (normalizedKeyRequirement.isPresent()) { + return keyProperty.satisfiesKeyRequirement(keyRequirement); + } + else { + return false; + } + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("KeyProperty", keyProperty) + .add("EquivalenceClassProperty", equivalenceClassProperty) + .add("MaxCardProperty", maxCardProperty) + .toString(); + } + + /** + * This logical properties builder should be used by PlanNode's that do not + * (yet perhaps) propagate or add logical properties. For example, a GroupIdNode does + * not propagate or add logical properties. This is the PlanNode default. + */ + public static class NoPropagateBuilder + { + KeyProperty keyProperty = new KeyProperty(); + MaxCardProperty maxCardProperty = new MaxCardProperty(); + EquivalenceClassProperty equivalenceClassProperty; + + NoPropagateBuilder(FunctionResolution functionResolution) + { + this.equivalenceClassProperty = new EquivalenceClassProperty(functionResolution); + } + + LogicalPropertiesImpl build() + { + return new LogicalPropertiesImpl(equivalenceClassProperty, maxCardProperty, keyProperty); + } + } + + /** + * This logical properties builder should be used by PlanNode's that simply + * propagate source properties without changes. For example, a SemiJoin node + * propagates the inputs of its non-filtering source without adding new properties. + * A SortNode also propagates the logical properties of its source without change. + */ + public static class PropagateBuilder + { + private final LogicalPropertiesImpl sourceProperties; + private final KeyProperty keyProperty = new KeyProperty(); + private final MaxCardProperty maxCardProperty = new MaxCardProperty(); + private final EquivalenceClassProperty equivalenceClassProperty; + + PropagateBuilder(LogicalPropertiesImpl sourceProperties, FunctionResolution functionResolution) + { + requireNonNull(sourceProperties, "sourceProperties is null"); + this.sourceProperties = sourceProperties; + this.equivalenceClassProperty = new EquivalenceClassProperty(functionResolution); + } + + LogicalPropertiesImpl build() + { + keyProperty.addKeys(sourceProperties.keyProperty); + maxCardProperty.update(sourceProperties.maxCardProperty); + equivalenceClassProperty.update(sourceProperties.equivalenceClassProperty); + return new LogicalPropertiesImpl(equivalenceClassProperty, maxCardProperty, keyProperty); + } + } + + /** + * This logical properties builder is used by a TableScanNode to initialize logical properties from catalog constraints. + */ + public static class TableScanBuilder + { + List> keys; + KeyProperty keyProperty = new KeyProperty(); + MaxCardProperty maxCardProperty = new MaxCardProperty(); + EquivalenceClassProperty equivalenceClassProperty; + + TableScanBuilder(List> keys, FunctionResolution functionResolution) + { + requireNonNull(keys, "keys is null"); + this.keys = keys; + this.equivalenceClassProperty = new EquivalenceClassProperty(functionResolution); + } + + LogicalPropertiesImpl build() + { + keyProperty.addKeys(keys.stream().map(keyCols -> new Key(keyCols)).collect(Collectors.toSet())); + return new LogicalPropertiesImpl(equivalenceClassProperty, maxCardProperty, keyProperty); + } + } + + /** + * This logical properties builder should be used by PlanNode's that apply predicates. + * The application of conjunct predicates that equate attributes and constants effects changes to the equivalence class property. + * When equivalence classes change, specifically when equivalence class heads change, properties that keep a canonical form + * in alignment with equivalence classes will be affected. + */ + public static class FilterBuilder + { + private final LogicalPropertiesImpl sourceProperties; + private final RowExpression predicate; + private final MaxCardProperty maxCardProperty = new MaxCardProperty(); + private final EquivalenceClassProperty equivalenceClassProperty; + private final KeyProperty keyProperty = new KeyProperty(); + + FilterBuilder(LogicalPropertiesImpl sourceProperties, RowExpression predicate, FunctionResolution functionResolution) + { + requireNonNull(sourceProperties, "sourceProperties is null"); + requireNonNull(predicate, "predicate is null"); + this.sourceProperties = sourceProperties; + this.predicate = predicate; + this.equivalenceClassProperty = new EquivalenceClassProperty(functionResolution); + } + + LogicalPropertiesImpl build() + { + keyProperty.addKeys(sourceProperties.keyProperty); + maxCardProperty.update(sourceProperties.maxCardProperty); + equivalenceClassProperty.update(sourceProperties.equivalenceClassProperty); + if (equivalenceClassProperty.update(predicate)) { + normalizeKeyPropertyAndSetMaxCard(keyProperty, maxCardProperty, equivalenceClassProperty); + } + return new LogicalPropertiesImpl(equivalenceClassProperty, maxCardProperty, keyProperty); + } + } + + /** + * This logical properties builder should be used by PlanNode's that project their + * source properties. For example, a ProjectNode and AggregationNode project their + * source properties. The former might also reassign property variable references. + */ + public static class ProjectBuilder + { + private final LogicalPropertiesImpl sourceProperties; + private final Assignments assignments; + private final MaxCardProperty maxCardProperty = new MaxCardProperty(); + private KeyProperty keyProperty = new KeyProperty(); + private EquivalenceClassProperty equivalenceClassProperty; + + ProjectBuilder(LogicalPropertiesImpl sourceProperties, Assignments assignments, FunctionResolution functionResolution) + { + requireNonNull(sourceProperties, "sourceProperties is null"); + requireNonNull(assignments, "assignments is null"); + this.sourceProperties = sourceProperties; + this.assignments = assignments; + this.equivalenceClassProperty = new EquivalenceClassProperty(functionResolution); + } + + LogicalPropertiesImpl build() + { + keyProperty.addKeys(sourceProperties.keyProperty); + maxCardProperty.update(sourceProperties.maxCardProperty); + equivalenceClassProperty.update(sourceProperties.equivalenceClassProperty); + + //project both equivalence classes and key property + Map inverseVariableAssignments = inverseVariableAssignments(assignments); + keyProperty = keyProperty.project(new InverseVariableMappingsWithEquivalence(equivalenceClassProperty, inverseVariableAssignments)); + equivalenceClassProperty = equivalenceClassProperty.project(inverseVariableAssignments); + normalizeKeyPropertyAndSetMaxCard(keyProperty, maxCardProperty, equivalenceClassProperty); + return new LogicalPropertiesImpl(equivalenceClassProperty, maxCardProperty, keyProperty); + } + } + + /** + * This logical properties builder should be used by PlanNode's that propagate their + * source properties and add a limit. For example, TopNNode and LimitNode. + */ + public static class PropagateAndLimitBuilder + { + private final LogicalPropertiesImpl sourceProperties; + private final long limit; + private final MaxCardProperty maxCardProperty = new MaxCardProperty(); + private final EquivalenceClassProperty equivalenceClassProperty; + private final KeyProperty keyProperty = new KeyProperty(); + + PropagateAndLimitBuilder(LogicalPropertiesImpl sourceProperties, long limit, FunctionResolution functionResolution) + { + requireNonNull(sourceProperties, "sourceProperties is null"); + this.sourceProperties = sourceProperties; + this.limit = limit; + this.equivalenceClassProperty = new EquivalenceClassProperty(functionResolution); + } + + LogicalPropertiesImpl build() + { + keyProperty.addKeys(sourceProperties.keyProperty); + maxCardProperty.update(sourceProperties.maxCardProperty); + maxCardProperty.update(limit); + equivalenceClassProperty.update(sourceProperties.equivalenceClassProperty); + if (maxCardProperty.isAtMostOne()) { + keyProperty.empty(); + } + return new LogicalPropertiesImpl(equivalenceClassProperty, maxCardProperty, keyProperty); + } + } + + /** + * This logical properties builder should be used by PlanNode's that propagate their source + * properties and add a unique key. For example, an AggregationNode with a single grouping key + * propagates it's input properties and adds the grouping key attributes as a new unique key. + * The resulting properties are projected by the provided output variables. + */ + public static class AggregationBuilder + { + private final LogicalPropertiesImpl sourceProperties; + private final Key key; + private final EquivalenceClassProperty equivalenceClassProperty; + private final List outputVariables; + private final KeyProperty keyProperty = new KeyProperty(); + private final MaxCardProperty maxCardProperty = new MaxCardProperty(); + private final FunctionResolution functionResolution; + + AggregationBuilder(LogicalPropertiesImpl sourceProperties, Set keyVariables, List outputVariables, FunctionResolution functionResolution) + { + requireNonNull(sourceProperties, "sourceProperties is null"); + requireNonNull(keyVariables, "keyVariables is null"); + requireNonNull(outputVariables, "outputVariables is null"); + checkArgument(!keyVariables.isEmpty(), "keyVariables is empty"); + this.sourceProperties = sourceProperties; + this.key = new Key(keyVariables); + this.outputVariables = outputVariables; + this.functionResolution = functionResolution; + this.equivalenceClassProperty = new EquivalenceClassProperty(functionResolution); + } + + LogicalPropertiesImpl build() + { + keyProperty.addKeys(sourceProperties.keyProperty); + maxCardProperty.update(sourceProperties.maxCardProperty); + equivalenceClassProperty.update(sourceProperties.equivalenceClassProperty); + //add the new key and normalize the key property unless there is a single row in the input + if (!maxCardProperty.isAtMostOne()) { + keyProperty.addKey(key); + normalizeKeyPropertyAndSetMaxCard(keyProperty, maxCardProperty, equivalenceClassProperty); + } + //project the properties using the output variables to ensure only the interesting constraints propagate + ProjectBuilder projectBuilder = new ProjectBuilder(new LogicalPropertiesImpl(equivalenceClassProperty, maxCardProperty, keyProperty), + AssignmentUtils.identityAssignments(this.outputVariables), functionResolution); + return projectBuilder.build(); + } + } + + /** + * This logical properties builder should be used by PlanNode's that propagate their source + * properties, add a unique key, and also limit the result. For example, a DistinctLimitNode. + */ + public static class DistinctLimitBuilder + { + private final LogicalPropertiesImpl sourceProperties; + private final Set keyVariables; + private final long limit; + private final List outputVariables; + private final FunctionResolution functionResolution; + + DistinctLimitBuilder(LogicalPropertiesImpl sourceProperties, Set keyVariables, Long limit, List outputVariables, FunctionResolution functionResolution) + { + requireNonNull(sourceProperties, "sourceProperties is null"); + requireNonNull(keyVariables, "keyVariables is null"); + requireNonNull(outputVariables, "outputVariables is null"); + requireNonNull(limit, "limit is null"); + checkArgument(!keyVariables.isEmpty(), "keyVariables is empty"); + this.sourceProperties = sourceProperties; + this.keyVariables = keyVariables; + this.outputVariables = outputVariables; + this.limit = limit; + this.functionResolution = functionResolution; + } + + LogicalPropertiesImpl build() + { + AggregationBuilder aggregationBuilder = new AggregationBuilder(sourceProperties, keyVariables, outputVariables, functionResolution); + PropagateAndLimitBuilder propagateAndLimitBuilder = new PropagateAndLimitBuilder(aggregationBuilder.build(), limit, functionResolution); + return propagateAndLimitBuilder.build(); + } + } + + /** + * This logical properties builder should be used by PlanNode's that join two input sources + * where both input sources contribute variables to the join output (e.g. JoinNode vs. SemiJoinNode). + * Propagation of the source properties of the join requires a sophisticated analysis of the characteristics of the join. + *

+ * Key and MaxCard Properties... + *

+ * - An inner or left join propagates the key property and maxcard property of the left source if the join is n-to-1, + * meaning that each row of the left source matches at most one row of the right source. Determining that a join is n-to-1 + * involves forming a key requirement from the equi-join attributes of the right table and querying the logical properties + * of the right table to determine if those attributes form a unique key. Semi-joins are inherently n-to1. + *

+ * - Conversely, an inner or right join can propagate the key property and maxcard property of the right source if the join is 1-to-n. + * If an inner join is 1-to-1, which is the case when it is both n-to-1 and 1-to-n, then it follows from the above that the key property + * of the join result comprises the union of the left source keys and right source keys. + *

+ * - If an inner join is instead m-to-n, meaning that it is neither n-to-1 nor 1-to-n, the key property of the join is formed by + * concatenating the left source and right source key properties. Concatenating two key properties forms a new key for every + * possible combination of keys. For example, if key property KP1 has key {A} and key {B,C} and key property KP2 has key {D} + * and key {E} the concatenating KP1 and KP2 would yield a key property with keys {A,D}, {A,E}, {B,C,D} and {B,C,E}. + * An m-to-n join propagates the product of the left source MaxCardProperty and right source MaxCardProperty if the values are both known. + *

+ * - Full outer joins do not propagate source key or maxcard properties as they can inject null rows into the result. + *

+ * EquivalenceClass Property .. + *

+ * - The equivalence class property of an inner or left join adds the equivalence classes of the left source. + *

+ * - The equivalence class property of an inner or right join adds the equivalence classes of the right source. + *

+ * - The equivalence class property of an inner join is then updated with any new equivalences resulting from the application of + * equi-join predicates, or equality conjuncts applied as filters. + *

+ * It follows from the above that inner joins combine the left and right source equivalence classes and that full outer joins do + * not propagate equivalence classes. + * Finally, the key property is normalized with the equivalence classes of the join, and both key and equivalence properties are + * projected with the join’s output attributes. + */ + public static class JoinBuilder + { + private final LogicalPropertiesImpl leftProperties; + private final LogicalPropertiesImpl rightProperties; + private final List equijoinPredicates; + private final JoinNode.Type joinType; + private final Optional filterPredicate; + private final List outputVariables; + private final MaxCardProperty maxCardProperty = new MaxCardProperty(); + private final EquivalenceClassProperty equivalenceClassProperty; + private final FunctionResolution functionResolution; + private KeyProperty keyProperty = new KeyProperty(); + + JoinBuilder(LogicalPropertiesImpl leftProperties, + LogicalPropertiesImpl rightProperties, + List equijoinPredicates, + JoinNode.Type joinType, + Optional filterPredicate, + List outputVariables, + FunctionResolution functionResolution) + { + requireNonNull(leftProperties, "leftProperties is null"); + requireNonNull(rightProperties, "rightProperties is null"); + requireNonNull(equijoinPredicates, "equijoinPredicates is null"); + requireNonNull(joinType, "joinType is null"); + requireNonNull(filterPredicate, "filterPredicate is null"); + requireNonNull(outputVariables, "outputVariables is null"); + this.leftProperties = leftProperties; + this.rightProperties = rightProperties; + this.equijoinPredicates = equijoinPredicates; + this.joinType = joinType; + this.filterPredicate = filterPredicate; + this.outputVariables = outputVariables; + this.functionResolution = functionResolution; + this.equivalenceClassProperty = new EquivalenceClassProperty(functionResolution); + } + + LogicalPropertiesImpl build() + { + // first determine if the join is n to 1 and/or 1 to n + boolean nToOne = false; + boolean oneToN = false; + Set rightJoinVariables = this.equijoinPredicates.stream().map(predicate -> predicate.getRight()).collect(Collectors.toSet()); + Set leftJoinVariables = this.equijoinPredicates.stream().map(predicate -> predicate.getLeft()).collect(Collectors.toSet()); + + //if n-to-1 inner or left join then propagate left source keys and maxcard + if ((rightProperties.maxCardProperty.isAtMostOne() || (!rightJoinVariables.isEmpty() && rightProperties.isDistinct(rightJoinVariables))) && + ((joinType == INNER || joinType == LEFT) || (joinType == FULL && leftProperties.maxCardProperty.isAtMost(1)))) { + nToOne = true; + keyProperty.addKeys(leftProperties.keyProperty); + maxCardProperty.update(leftProperties.maxCardProperty); + } + + //if 1-to-n inner or right join then propagate right source keys and maxcard + if ((leftProperties.maxCardProperty.isAtMostOne() || (!leftJoinVariables.isEmpty() && leftProperties.isDistinct(leftJoinVariables))) && + ((joinType == INNER || joinType == RIGHT) || (joinType == FULL && rightProperties.maxCardProperty.isAtMost(1)))) { + oneToN = true; + keyProperty.addKeys(rightProperties.keyProperty); + maxCardProperty.update(rightProperties.maxCardProperty); + } + + //if an n-to-m then multiply maxcards and, if inner join, concatenate keys + if (!(nToOne || oneToN)) { + maxCardProperty.update(leftProperties.maxCardProperty); + maxCardProperty.multiply(rightProperties.maxCardProperty); + if (joinType == INNER) { + keyProperty.addKeys(leftProperties.keyProperty); + keyProperty = keyProperty.concat(rightProperties.keyProperty); + } + } + + //propagate left source equivalence classes if nulls cannot be injected + if (joinType == INNER || joinType == LEFT) { + equivalenceClassProperty.update(leftProperties.equivalenceClassProperty); + } + + //propagate right source equivalence classes if nulls cannot be injected + if (joinType == INNER || joinType == RIGHT) { + equivalenceClassProperty.update(rightProperties.equivalenceClassProperty); + } + + //update equivalence classes with equijoin predicates, note that if nulls are injected, equivalence does not hold propagate + if (joinType == INNER) { + equijoinPredicates.stream().forEach(joinVariables -> equivalenceClassProperty.update(joinVariables.getLeft(), joinVariables.getRight())); + + //update equivalence classes with any residual filter predicate + if (filterPredicate.isPresent()) { + equivalenceClassProperty.update(filterPredicate.get()); + } + } + + //since we likely merged equivalence class from left and right source we will normalize the key property + normalizeKeyPropertyAndSetMaxCard(keyProperty, maxCardProperty, equivalenceClassProperty); + + //project the resulting properties by the output variables + ProjectBuilder projectBuilder = new ProjectBuilder(new LogicalPropertiesImpl(equivalenceClassProperty, maxCardProperty, keyProperty), + AssignmentUtils.identityAssignments(this.outputVariables), functionResolution); + return projectBuilder.build(); + } + } + + /** + * This is a helper method for project operations where variable references are reassigned. + * It uses equivalence classes to facilitate the reassignment. For example, if a key + * is normalized to equivalence class head X with equivalence class member Y and there is a reassignment + * of Y to YY then the variable X will be reassigned to YY assuming there is no direct + * reassignment of X to another variable reference. Useful equivalent mappings are + * determined lazily and cached. + */ + public static class InverseVariableMappingsWithEquivalence + { + private final EquivalenceClassProperty equivalenceClassProperty; + private final Map inverseMappings; + + InverseVariableMappingsWithEquivalence(EquivalenceClassProperty equivalenceClassProperty, + Map inverseMappings) + { + requireNonNull(equivalenceClassProperty, "equivalenceClassProperty is null"); + requireNonNull(inverseMappings, "inverseMappings is null"); + this.equivalenceClassProperty = equivalenceClassProperty; + this.inverseMappings = inverseMappings; + } + + private boolean containsKey(VariableReferenceExpression variable) + { + if (!inverseMappings.containsKey(variable)) { + //try to find a reverse mapping of an equivalent variable, update mappings + RowExpression head = equivalenceClassProperty.getEquivalenceClassHead(variable); + List equivalentVariables = new ArrayList<>(); + equivalentVariables.add(head); + equivalentVariables.addAll(equivalenceClassProperty.getEquivalenceClasses(head)); + for (RowExpression e : equivalentVariables) { + if (e instanceof VariableReferenceExpression && + inverseMappings.containsKey(e)) { + inverseMappings.put(variable, inverseMappings.get(e)); + break; + } + } + } + return inverseMappings.containsKey(variable); + } + + /** + * Returns a direct or equivalent mapping of the provided variable reference. + */ + public Optional get(VariableReferenceExpression variable) + { + requireNonNull(variable, "variable is null"); + if (containsKey(variable)) { + return Optional.of(inverseMappings.get(variable)); + } + else { + return Optional.empty(); + } + } + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/LogicalPropertiesProviderImpl.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/LogicalPropertiesProviderImpl.java new file mode 100644 index 0000000000000..a12a3b439dcf0 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/LogicalPropertiesProviderImpl.java @@ -0,0 +1,345 @@ +/* + * 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 com.facebook.presto.sql.planner.iterative.properties; + +import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.constraints.TableConstraint; +import com.facebook.presto.spi.constraints.UniqueConstraint; +import com.facebook.presto.spi.plan.AggregationNode; +import com.facebook.presto.spi.plan.DistinctLimitNode; +import com.facebook.presto.spi.plan.FilterNode; +import com.facebook.presto.spi.plan.LimitNode; +import com.facebook.presto.spi.plan.LogicalProperties; +import com.facebook.presto.spi.plan.LogicalPropertiesProvider; +import com.facebook.presto.spi.plan.PlanNode; +import com.facebook.presto.spi.plan.ProjectNode; +import com.facebook.presto.spi.plan.TableScanNode; +import com.facebook.presto.spi.plan.TopNNode; +import com.facebook.presto.spi.plan.ValuesNode; +import com.facebook.presto.spi.relation.VariableReferenceExpression; +import com.facebook.presto.sql.planner.iterative.GroupReference; +import com.facebook.presto.sql.planner.plan.AssignUniqueId; +import com.facebook.presto.sql.planner.plan.JoinNode; +import com.facebook.presto.sql.planner.plan.SemiJoinNode; +import com.facebook.presto.sql.planner.plan.SortNode; +import com.facebook.presto.sql.relational.FunctionResolution; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import static java.util.Objects.requireNonNull; + +/** + * Defines a suite of plan node-specific methods for the computation of logical properties. + * Supplies a default implementation that produces an empty set of logical properties, and additionally, + * a suite of plan-node specific overrides of the default implementation. The implementations leverage + * property propagation builders supplied by LogicalPropertiesImpl. The LogicalPropertiesProvider + * mechanism enables a plan node to receive its logical property compute capabilities via dependency injection. + * This is needed because the computation of logical properties requires analysis of plan node's argument expressions, + * and the code the performs this analysis must be built in presto-main as this is where expression related classes are visible. + * The property computation implementation is dynamically injected into the presto-spi and presto-main plan node method's responsible + * for computing logical properties. + */ +public class LogicalPropertiesProviderImpl + implements LogicalPropertiesProvider +{ + private final FunctionResolution functionResolution; + + public LogicalPropertiesProviderImpl(FunctionResolution functionResolution) + { + this.functionResolution = requireNonNull(functionResolution, "functionResolution is null"); + } + + /** + * Provides the logical properties for a ValuesNode. Bounds the MaxCard property to the row count. + * + * @param valuesNode + * @return The logical properties for a ValuesNode. + */ + @Override + public LogicalProperties getValuesProperties(ValuesNode valuesNode) + { + LogicalPropertiesImpl sourceProps = new LogicalPropertiesImpl.NoPropagateBuilder(functionResolution).build(); + LogicalPropertiesImpl.PropagateAndLimitBuilder propagateAndLimitBuilder = new LogicalPropertiesImpl.PropagateAndLimitBuilder(sourceProps, valuesNode.getRows().size(), functionResolution); + return propagateAndLimitBuilder.build(); + } + + /** + * Provides the logical properties for a TableScanNode. These properties come from analysis of catalog constraints. + * The current implementation is just scaffolding that will be removed once the HMS upgrade is completed. + * + * @param tableScanNode + * @return The logical properties for a TableScanNode. + */ + @Override + public LogicalProperties getTableScanProperties(TableScanNode tableScanNode) + { + // map primary key and unique constraints from column handles to variable reference expressions + List> keys = new ArrayList<>(); + List> uniqueConstraints = tableScanNode.getTableConstraints().stream().filter(tableConstraint -> tableConstraint instanceof UniqueConstraint && (tableConstraint.isEnforced() || tableConstraint.isRely())).collect(Collectors.toList()); + if (!uniqueConstraints.isEmpty()) { + Map assignments = tableScanNode.getAssignments(); + Map inverseAssignments = assignments.entrySet().stream().collect(Collectors.toMap(Map.Entry::getValue, Map.Entry::getKey)); + uniqueConstraints.stream().filter(uniqueConstraint -> uniqueConstraint.getColumns().stream().allMatch(col -> inverseAssignments.containsKey(col))).forEach(uniqueConstraint -> keys.add(uniqueConstraint.getColumns().stream().map(col -> inverseAssignments.get(col)).collect(Collectors.toSet()))); + } + LogicalPropertiesImpl.TableScanBuilder logicalPropsBuilder = new LogicalPropertiesImpl.TableScanBuilder(keys, functionResolution); + return logicalPropsBuilder.build(); + } + + /** + * Provides the logical properties for a FilterNode. These properties reflect the effects of applying predicates to the source properties. + * + * @param filterNode + * @return The logical properties for a FilterNode. + */ + @Override + public LogicalProperties getFilterProperties(FilterNode filterNode) + { + if (!((filterNode.getSource() instanceof GroupReference) && ((GroupReference) filterNode.getSource()).getLogicalProperties().isPresent())) { + throw new IllegalStateException("Expected source PlanNode to be a GroupReference with LogicalProperties"); + } + LogicalPropertiesImpl sourceProps = (LogicalPropertiesImpl) ((GroupReference) filterNode.getSource()).getLogicalProperties().get(); + LogicalPropertiesImpl.FilterBuilder logicalPropsBuilder = new LogicalPropertiesImpl.FilterBuilder(sourceProps, filterNode.getPredicate(), functionResolution); + return logicalPropsBuilder.build(); + } + + /** + * Provides the logical properties for a ProjectNode. These properties are essentially a projection and reassignment of the + * variable references in the source properties. + * + * @param projectNode + * @return The logical properties for a ProjectNode. + */ + @Override + public LogicalProperties getProjectProperties(ProjectNode projectNode) + { + if (!((projectNode.getSource() instanceof GroupReference) && ((GroupReference) projectNode.getSource()).getLogicalProperties().isPresent())) { + throw new IllegalStateException("Expected source PlanNode to be a GroupReference with LogicalProperties"); + } + LogicalPropertiesImpl sourceProps = (LogicalPropertiesImpl) ((GroupReference) projectNode.getSource()).getLogicalProperties().get(); + LogicalPropertiesImpl.ProjectBuilder logicalPropsBuilder = new LogicalPropertiesImpl.ProjectBuilder(sourceProps, projectNode.getAssignments(), functionResolution); + return logicalPropsBuilder.build(); + } + + /** + * Provides the logical properties for a JoinNode. These properties reflect the effects of combining the properties of the left and right sources. + * + * @param node An instance of JoinNode. + * @return The logical properties for a JoinNode. + */ + @Override + public LogicalProperties getJoinProperties(PlanNode node) + { + if (!(node instanceof JoinNode)) { + throw new IllegalArgumentException("Expected PlanNode to be instance of JoinNode"); + } + + JoinNode joinNode = (JoinNode) node; + if (!((joinNode.getLeft() instanceof GroupReference) && ((GroupReference) joinNode.getLeft()).getLogicalProperties().isPresent())) { + throw new IllegalStateException("Expected left source PlanNode to be a GroupReference with LogicalProperties"); + } + + if (!((joinNode.getRight() instanceof GroupReference) && ((GroupReference) joinNode.getRight()).getLogicalProperties().isPresent())) { + throw new IllegalStateException("Expected right source PlanNode to be a GroupReference with LogicalProperties"); + } + + LogicalPropertiesImpl leftProps = (LogicalPropertiesImpl) ((GroupReference) joinNode.getLeft()).getLogicalProperties().get(); + LogicalPropertiesImpl rightProps = (LogicalPropertiesImpl) ((GroupReference) joinNode.getRight()).getLogicalProperties().get(); + LogicalPropertiesImpl.JoinBuilder logicalPropsBuilder = new LogicalPropertiesImpl.JoinBuilder(leftProps, rightProps, joinNode.getCriteria(), joinNode.getType(), joinNode.getFilter(), joinNode.getOutputVariables(), functionResolution); + return logicalPropsBuilder.build(); + } + + /** + * Provides the logical properties for a SemiJoinNode. The properties of the non-filtering source are propagated without change. + * + * @param node An instance of SemiJoinNode. + * @return The logical properties for a SemiJoinNode. + */ + @Override + public LogicalProperties getSemiJoinProperties(PlanNode node) + { + if (!(node instanceof SemiJoinNode)) { + throw new IllegalArgumentException("Expected PlanNode to be instance of SemiJoinNode"); + } + + SemiJoinNode semiJoinNode = (SemiJoinNode) node; + if (!((semiJoinNode.getSource() instanceof GroupReference) && ((GroupReference) semiJoinNode.getSource()).getLogicalProperties().isPresent())) { + throw new IllegalStateException("Expected non-filtering source PlanNode to be a GroupReference with LogicalProperties"); + } + + LogicalPropertiesImpl sourceProps = (LogicalPropertiesImpl) ((GroupReference) semiJoinNode.getSource()).getLogicalProperties().get(); + LogicalPropertiesImpl.PropagateBuilder propagateBuilder = new LogicalPropertiesImpl.PropagateBuilder(sourceProps, functionResolution); + return propagateBuilder.build(); + } + + /** + * Provides the logical properties for an AggregationNode. If a distinct or grouping operation is being performed then the logical properties + * reflect the addition of a unique key to the source properties. + * + * @param aggregationNode + * @return The logical properties for an AggregationNode. + */ + @Override + public LogicalProperties getAggregationProperties(AggregationNode aggregationNode) + { + if (!((aggregationNode.getSource() instanceof GroupReference) && ((GroupReference) aggregationNode.getSource()).getLogicalProperties().isPresent())) { + throw new IllegalStateException("Expected source PlanNode to be a GroupReference with LogicalProperties"); + } + + if (aggregationNode.getGroupingKeys().isEmpty() && aggregationNode.getAggregations().isEmpty()) { + throw new IllegalStateException("Aggregation node with no grouping columns and no aggregation functions"); + } + + LogicalPropertiesImpl sourceProps = (LogicalPropertiesImpl) ((GroupReference) aggregationNode.getSource()).getLogicalProperties().get(); + if (!aggregationNode.getAggregations().isEmpty() && aggregationNode.getGroupingKeys().isEmpty()) { + //aggregation with no grouping variables, single row output + LogicalPropertiesImpl.PropagateAndLimitBuilder propagateBuilder = new LogicalPropertiesImpl.PropagateAndLimitBuilder(sourceProps, Long.valueOf(1), functionResolution); + return propagateBuilder.build(); + } + else { + LogicalPropertiesImpl.AggregationBuilder aggregationBuilder = new LogicalPropertiesImpl.AggregationBuilder(sourceProps, + aggregationNode.getGroupingKeys().stream().collect(Collectors.toSet()), + aggregationNode.getOutputVariables(), + functionResolution); + return aggregationBuilder.build(); + } + } + + /** + * Provides the logical properties for a AssignUniqueId. Adds the id variable as a key. + * + * @param node + * @return The logical properties for a AssignUniqueId. + */ + @Override + public LogicalProperties getAssignUniqueIdProperties(PlanNode node) + { + if (!(node instanceof AssignUniqueId)) { + throw new IllegalArgumentException("Expected PlanNode to be instance of AssignUniqueId"); + } + + AssignUniqueId assignUniqueIdNode = (AssignUniqueId) node; + if (!((assignUniqueIdNode.getSource() instanceof GroupReference) && ((GroupReference) assignUniqueIdNode.getSource()).getLogicalProperties().isPresent())) { + throw new IllegalStateException("Expected source PlanNode to be a GroupReference with LogicalProperties"); + } + + if (assignUniqueIdNode.getIdVariable() == null) { + throw new IllegalStateException("AssignUniqueId should have an id variable"); + } + + LogicalPropertiesImpl sourceProps = (LogicalPropertiesImpl) ((GroupReference) assignUniqueIdNode.getSource()).getLogicalProperties().get(); + Set key = new HashSet<>(); + key.add(assignUniqueIdNode.getIdVariable()); + LogicalPropertiesImpl.AggregationBuilder aggregationBuilder = new LogicalPropertiesImpl.AggregationBuilder(sourceProps, key, assignUniqueIdNode.getOutputVariables(), functionResolution); + return aggregationBuilder.build(); + } + + /** + * Provides the logical properties for a DistinctLimitNode. The resulting properties reflect the application of both a distinct and limit N to the + * source properties. The distinct aspect adds a unique key to the source properties and the limit bounds the result to N. + * + * @param distinctLimitNode + * @return The logical properties for an DistinctLimitNode. + */ + @Override + public LogicalProperties getDistinctLimitProperties(DistinctLimitNode distinctLimitNode) + { + if (!((distinctLimitNode.getSource() instanceof GroupReference) && ((GroupReference) distinctLimitNode.getSource()).getLogicalProperties().isPresent())) { + throw new IllegalStateException("Expected source PlanNode to be a GroupReference with LogicalProperties"); + } + + LogicalPropertiesImpl sourceProps = (LogicalPropertiesImpl) ((GroupReference) distinctLimitNode.getSource()).getLogicalProperties().get(); + LogicalPropertiesImpl.DistinctLimitBuilder aggregationBuilder = new LogicalPropertiesImpl.DistinctLimitBuilder(sourceProps, + distinctLimitNode.getDistinctVariables().stream().collect(Collectors.toSet()), + distinctLimitNode.getLimit(), + distinctLimitNode.getOutputVariables(), + functionResolution); + return aggregationBuilder.build(); + } + + /** + * Provides the logical properties for a LimitNode. The properties reflect the application of a limit N to the source properties. + * + * @param limitNode + * @return The logical properties for a LimitNode. + */ + @Override + public LogicalProperties getLimitProperties(LimitNode limitNode) + { + if (!((limitNode.getSource() instanceof GroupReference) && ((GroupReference) limitNode.getSource()).getLogicalProperties().isPresent())) { + throw new IllegalStateException("Expected source PlanNode to be a GroupReference with LogicalProperties"); + } + + LogicalPropertiesImpl sourceProps = (LogicalPropertiesImpl) ((GroupReference) limitNode.getSource()).getLogicalProperties().get(); + LogicalPropertiesImpl.PropagateAndLimitBuilder propagateBuilder = new LogicalPropertiesImpl.PropagateAndLimitBuilder(sourceProps, limitNode.getCount(), functionResolution); + return propagateBuilder.build(); + } + + /** + * Provides the logical properties for a LimitNode. The properties reflect the application of a limit N to the source properties. + * + * @param topNNode + * @return The logical properties for a LimitNode. + */ + @Override + public LogicalProperties getTopNProperties(TopNNode topNNode) + { + if (!((topNNode.getSource() instanceof GroupReference) && ((GroupReference) topNNode.getSource()).getLogicalProperties().isPresent())) { + throw new IllegalStateException("Expected left source PlanNode to be a GroupReference with LogicalProperties"); + } + + LogicalPropertiesImpl sourceProps = (LogicalPropertiesImpl) ((GroupReference) topNNode.getSource()).getLogicalProperties().get(); + LogicalPropertiesImpl.PropagateAndLimitBuilder propagateBuilder = new LogicalPropertiesImpl.PropagateAndLimitBuilder(sourceProps, topNNode.getCount(), functionResolution); + return propagateBuilder.build(); + } + + /** + * Provides the logical properties for a SortNode. The properties of the source are propagated without change. + * + * @param node An instance of SortNode. + * @return The logical properties for a SortNode. + */ + @Override + public LogicalProperties getSortProperties(PlanNode node) + { + if (!(node instanceof SortNode)) { + throw new IllegalArgumentException("Expected PlanNode to be instance of SortNode"); + } + + SortNode sortNode = (SortNode) node; + if (!((sortNode.getSource() instanceof GroupReference) && ((GroupReference) sortNode.getSource()).getLogicalProperties().isPresent())) { + throw new IllegalStateException("Expected source PlanNode to be a GroupReference with LogicalProperties"); + } + + LogicalPropertiesImpl sourceProps = (LogicalPropertiesImpl) ((GroupReference) sortNode.getSource()).getLogicalProperties().get(); + LogicalPropertiesImpl.PropagateBuilder propagateBuilder = new LogicalPropertiesImpl.PropagateBuilder(sourceProps, functionResolution); + return propagateBuilder.build(); + } + + /** + * Provides the default logical properties for a generic PlanNode which is essentially an empty set of properties. + * + * @return The default set of logical properties for a generic PlanNode. + */ + @Override + public LogicalProperties getDefaultProperties() + { + LogicalPropertiesImpl.NoPropagateBuilder logicalPropsBuilder = new LogicalPropertiesImpl.NoPropagateBuilder(functionResolution); + return logicalPropsBuilder.build(); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/MaxCardProperty.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/MaxCardProperty.java new file mode 100644 index 0000000000000..4c4d6eb980c03 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/MaxCardProperty.java @@ -0,0 +1,137 @@ +/* + * 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 com.facebook.presto.sql.planner.iterative.properties; + +import java.util.Optional; + +import static com.google.common.base.MoreObjects.toStringHelper; + +/** + * Represents a provable maximum number of rows in a final or intermediate result by a PlanNode. + * For example if a unique key is fully bound to constants by predicates the maxcard property + * can be set to one. A limit or top operation might set maxcard to the value of their count argument. + * The value is unknown until determined and set. + */ +public class MaxCardProperty +{ + private Optional value; + + public MaxCardProperty() + { + this.value = Optional.empty(); + } + + public MaxCardProperty(Long value) + { + this.value = Optional.of(value); + } + + /** + * True if this maxcard is more general than another. This is the case if a) neither maxcard is known or + * b) this maxcard is known and the other maxcard is either unknown or is set to larger or equal value. + * Note that two maxcard properties are equal if each is more general than the other. + * + * @param otherMaxCardProperty + * @return True if maxCardProperty is more general than otherMaxCardProperty or False otherwise. + */ + public boolean moreGeneral(MaxCardProperty otherMaxCardProperty) + { + return (!value.isPresent() && !otherMaxCardProperty.value.isPresent()) || + (value.isPresent() && (!otherMaxCardProperty.value.isPresent() || (otherMaxCardProperty.value.get() >= value.get()))); + } + + /** + * Updates this maxcard with the provided value. Will change the current value only if the current value is unknown + * or the provided value is known and smaller than the current setting. + * + * @param value + */ + public void update(long value) + { + if (!this.value.isPresent() || this.value.get().compareTo(value) > 0) { + this.value = Optional.of(value); + } + } + + /** + * Updates this maxcard with the provided maxcard property. Will change the current value only if the current value is unknown + * or the provided value is known and smaller than the current setting. + * + * @param sourceMaxCardProperty + */ + public void update(MaxCardProperty sourceMaxCardProperty) + { + if (sourceMaxCardProperty.value.isPresent()) { + if (this.value.isPresent()) { + this.value = Optional.of(Long.min(this.value.get(), sourceMaxCardProperty.value.get())); + } + else { + this.value = Optional.of(sourceMaxCardProperty.value.get()); + } + } + } + + /** + * True if maxcard is known and set to the value 1. + * This guarantee can be used to eliminate redundant sorts, distincts, topN's, etc. + * + * @return True if maxcard is set and to the value 1. + */ + public boolean isAtMostOne() + { + return this.isAtMost(1); + } + + /** + * True if maxcard is known and is at most n. + * + * @return True if maxcard is known and is at most n. + */ + public boolean isAtMost(long n) + { + if (value.isPresent()) { + return (value.get().longValue() <= n); + } + else { + return false; + } + } + + /** + * Performs the product of this maxcard and a provided maxcard if both have known values. + * Used to compute the maxcard of a join. + * + * @param maxCardProperty + */ + public void multiply(MaxCardProperty maxCardProperty) + { + //the product of empty and anything else is empty + if (!maxCardProperty.value.isPresent()) { + this.value = Optional.empty(); + return; + } + //new value is present and so multiply the current value if it is present + if (this.value.isPresent()) { + this.value = Optional.of(Long.valueOf(this.value.get() * maxCardProperty.value.get())); + } + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("value", (this.value.isPresent() ? value.get() : "null")) + .toString(); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/MergeLimitWithDistinct.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/MergeLimitWithDistinct.java index 0ffc3d447cba1..e8c9ab547a454 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/MergeLimitWithDistinct.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/MergeLimitWithDistinct.java @@ -33,17 +33,7 @@ public class MergeLimitWithDistinct private static final Pattern PATTERN = limit() .with(source().matching(aggregation().capturedAs(CHILD) - .matching(MergeLimitWithDistinct::isDistinct))); - - /** - * Whether this node corresponds to a DISTINCT operation in SQL - */ - private static boolean isDistinct(AggregationNode node) - { - return node.getAggregations().isEmpty() && - node.getOutputVariables().size() == node.getGroupingKeys().size() && - node.getOutputVariables().containsAll(node.getGroupingKeys()); - } + .matching(AggregationNode::isDistinct))); @Override public Pattern getPattern() diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PickTableLayout.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PickTableLayout.java index a7b96b0fea897..f676e9c45c51d 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PickTableLayout.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PickTableLayout.java @@ -226,6 +226,7 @@ public Result apply(TableScanNode tableScanNode, Captures captures, Context cont layout.getLayout().getNewTableHandle(), tableScanNode.getOutputVariables(), tableScanNode.getAssignments(), + tableScanNode.getTableConstraints(), layout.getLayout().getPredicate(), TupleDomain.all())); } @@ -321,6 +322,7 @@ private static PlanNode pushPredicateIntoTableScan( layout.getLayout().getNewTableHandle(), node.getOutputVariables(), node.getAssignments(), + node.getTableConstraints(), layout.getLayout().getPredicate(), computeEnforced(newDomain, layout.getUnenforcedConstraint())); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneTableScanColumns.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneTableScanColumns.java index 656c40ac5e32f..ec0f875273c53 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneTableScanColumns.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneTableScanColumns.java @@ -44,6 +44,7 @@ protected Optional pushDownProjectOff(PlanNodeIdAllocator idAllocator, tableScanNode.getTable(), filteredCopy(tableScanNode.getOutputVariables(), referencedOutputs::contains), filterKeys(tableScanNode.getAssignments(), referencedOutputs::contains), + tableScanNode.getTableConstraints(), tableScanNode.getCurrentConstraint(), tableScanNode.getEnforcedConstraint())); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RemoveRedundantAggregateDistinct.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RemoveRedundantAggregateDistinct.java new file mode 100644 index 0000000000000..b4992bf60a9ea --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RemoveRedundantAggregateDistinct.java @@ -0,0 +1,80 @@ +/* + * 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 com.facebook.presto.sql.planner.iterative.rule; + +import com.facebook.presto.matching.Captures; +import com.facebook.presto.matching.Pattern; +import com.facebook.presto.spi.plan.AggregationNode; +import com.facebook.presto.spi.relation.VariableReferenceExpression; +import com.facebook.presto.sql.planner.iterative.GroupReference; +import com.facebook.presto.sql.planner.iterative.Rule; + +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static com.facebook.presto.spi.plan.AggregationNode.Aggregation.removeDistinct; +import static com.facebook.presto.sql.planner.plan.Patterns.aggregation; + +/** + * Removes distinct from aggregates where the combination of aggregate columns and grouping variables contain a unique key. + * Ultimately this optimization needs to happen before the mark distinct optimization occurs. + * This will require moving the operations that transform away original expressions earlier in the sequence + * as logical property computation is designed to sit behind that transformation. For now this rule + * can be tested by disabling the mark distinct rule. + */ +public class RemoveRedundantAggregateDistinct + implements Rule +{ + private static final Pattern PATTERN = aggregation() + .matching(RemoveRedundantAggregateDistinct::aggregateDistinctOfUniqueKey); + + private static boolean aggregateDistinctOfUniqueKey(AggregationNode node) + { + return !node.getAggregations().isEmpty() && + ((GroupReference) node.getSource()).getLogicalProperties().isPresent() && + node.getAggregations().values().stream() + .filter(AggregationNode.Aggregation::isDistinct) + .anyMatch(a -> ((GroupReference) node.getSource()).getLogicalProperties().get().isDistinct( + Stream.concat(node.getGroupingKeys().stream().map(VariableReferenceExpression.class::cast), + a.getArguments().stream().map(VariableReferenceExpression.class::cast)).collect(Collectors.toSet()))); + } + + @Override + public Pattern getPattern() + { + return PATTERN; + } + + @Override + public Result apply(AggregationNode node, Captures captures, Context context) + { + //create new AggregateNode same as original but with distinct turned off for + //any aggregate function whose argument variables + grouping variables form a unique key + return Result.ofPlanNode(new AggregationNode( + node.getSourceLocation(), + context.getIdAllocator().getNextId(), + node.getSource(), + node.getAggregations().entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> + (e.getValue().isDistinct() && + ((GroupReference) node.getSource()).getLogicalProperties().get().isDistinct( + Stream.concat(node.getGroupingKeys().stream().map(VariableReferenceExpression.class::cast), + (e.getValue()).getArguments().stream().map(VariableReferenceExpression.class::cast)).collect(Collectors.toSet()))) ? + removeDistinct(e.getValue()) : (e.getValue()))), + node.getGroupingSets(), + node.getPreGroupedVariables(), + node.getStep(), + node.getHashVariable(), + node.getGroupIdVariable())); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RemoveRedundantDistinct.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RemoveRedundantDistinct.java new file mode 100644 index 0000000000000..2efa4f492a434 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RemoveRedundantDistinct.java @@ -0,0 +1,55 @@ +/* + * 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 com.facebook.presto.sql.planner.iterative.rule; + +import com.facebook.presto.matching.Captures; +import com.facebook.presto.matching.Pattern; +import com.facebook.presto.spi.plan.AggregationNode; +import com.facebook.presto.sql.planner.iterative.GroupReference; +import com.facebook.presto.sql.planner.iterative.Rule; + +import java.util.stream.Collectors; + +import static com.facebook.presto.sql.planner.plan.Patterns.aggregation; + +/** + * Removes distinct operations where the grouping variables contain a unique key. + */ +public class RemoveRedundantDistinct + implements Rule +{ + private static final Pattern PATTERN = aggregation() + .matching(AggregationNode::isDistinct) + .matching(RemoveRedundantDistinct::distinctOfUniqueKey); + + private static boolean distinctOfUniqueKey(AggregationNode node) + { + return node.hasNonEmptyGroupingSet() && + node.getAggregations().isEmpty() && + ((GroupReference) node.getSource()).getLogicalProperties().isPresent() && + ((GroupReference) node.getSource()).getLogicalProperties().get().isDistinct(node.getGroupingKeys().stream().collect(Collectors.toSet())); + } + + @Override + public Pattern getPattern() + { + return PATTERN; + } + + @Override + public Result apply(AggregationNode node, Captures captures, Context context) + { + return Result.ofPlanNode(node.getSource()); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RemoveRedundantDistinctLimit.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RemoveRedundantDistinctLimit.java new file mode 100644 index 0000000000000..bf02c806812d3 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RemoveRedundantDistinctLimit.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 com.facebook.presto.sql.planner.iterative.rule; + +import com.facebook.presto.matching.Captures; +import com.facebook.presto.matching.Pattern; +import com.facebook.presto.spi.plan.DistinctLimitNode; +import com.facebook.presto.sql.planner.iterative.GroupReference; +import com.facebook.presto.sql.planner.iterative.Rule; + +import static com.google.common.base.Preconditions.checkArgument; + +public class RemoveRedundantDistinctLimit + implements Rule +{ + private static final Pattern PATTERN = Pattern.typeOf(DistinctLimitNode.class) + .matching(RemoveRedundantDistinctLimit::singleRowInput); + + private static boolean singleRowInput(DistinctLimitNode node) + { + return (((GroupReference) node.getSource()).getLogicalProperties().isPresent() && + ((GroupReference) node.getSource()).getLogicalProperties().get().isAtMostSingleRow()); + } + + @Override + public Pattern getPattern() + { + return PATTERN; + } + + @Override + public Result apply(DistinctLimitNode node, Captures captures, Context context) + { + checkArgument(!node.getHashVariable().isPresent(), "HashSymbol should be empty"); + return Result.ofPlanNode(node.getSource()); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RemoveRedundantLimit.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RemoveRedundantLimit.java new file mode 100644 index 0000000000000..c0fa7fec4de64 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RemoveRedundantLimit.java @@ -0,0 +1,51 @@ +/* + * 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 com.facebook.presto.sql.planner.iterative.rule; + +import com.facebook.presto.matching.Captures; +import com.facebook.presto.matching.Pattern; +import com.facebook.presto.spi.plan.LimitNode; +import com.facebook.presto.sql.planner.iterative.GroupReference; +import com.facebook.presto.sql.planner.iterative.Rule; + +import static com.facebook.presto.sql.planner.plan.Patterns.limit; + +/** + * Remove Limit node when the subplan is guaranteed to produce fewer rows than the limit. + */ +public class RemoveRedundantLimit + implements Rule +{ + // Applies to both LimitNode with ties and LimitNode without ties. + private static final Pattern PATTERN = limit() + .matching(RemoveRedundantLimit::isAtMost); + + @Override + public Pattern getPattern() + { + return PATTERN; + } + + private static boolean isAtMost(LimitNode node) + { + return ((GroupReference) node.getSource()).getLogicalProperties().isPresent() && + ((GroupReference) node.getSource()).getLogicalProperties().get().isAtMost(node.getCount()); + } + + @Override + public Result apply(LimitNode limit, Captures captures, Context context) + { + return Result.ofPlanNode(limit.getSource()); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RemoveRedundantSort.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RemoveRedundantSort.java new file mode 100644 index 0000000000000..2de59ead98553 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RemoveRedundantSort.java @@ -0,0 +1,50 @@ +/* + * 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 com.facebook.presto.sql.planner.iterative.rule; + +import com.facebook.presto.matching.Captures; +import com.facebook.presto.matching.Pattern; +import com.facebook.presto.sql.planner.iterative.GroupReference; +import com.facebook.presto.sql.planner.iterative.Rule; +import com.facebook.presto.sql.planner.plan.SortNode; + +import static com.facebook.presto.sql.planner.plan.Patterns.sort; + +/** + * Removes sort operations where the input is provably at most one row. + */ +public class RemoveRedundantSort + implements Rule +{ + private static final Pattern PATTERN = sort() + .matching(RemoveRedundantSort::singleRowInput); + + private static boolean singleRowInput(SortNode node) + { + return ((GroupReference) node.getSource()).getLogicalProperties().isPresent() && + ((GroupReference) node.getSource()).getLogicalProperties().get().isAtMostSingleRow(); + } + + @Override + public Pattern getPattern() + { + return PATTERN; + } + + @Override + public Result apply(SortNode node, Captures captures, Context context) + { + return Result.ofPlanNode(node.getSource()); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RemoveRedundantTopN.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RemoveRedundantTopN.java new file mode 100644 index 0000000000000..a08bc6a3e94fc --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RemoveRedundantTopN.java @@ -0,0 +1,51 @@ +/* + * 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 com.facebook.presto.sql.planner.iterative.rule; + +import com.facebook.presto.matching.Captures; +import com.facebook.presto.matching.Pattern; +import com.facebook.presto.spi.plan.TopNNode; +import com.facebook.presto.sql.planner.iterative.GroupReference; +import com.facebook.presto.sql.planner.iterative.Rule; + +import static com.facebook.presto.sql.planner.plan.Patterns.topN; + +/** + * Removes top N operations where the input is provably at most one row. + */ +public class RemoveRedundantTopN + implements Rule +{ + private static final Pattern PATTERN = topN() + .matching(RemoveRedundantTopN::singleRowInput); + + private static boolean singleRowInput(TopNNode node) + { + return (node.getStep() == TopNNode.Step.SINGLE && + ((GroupReference) node.getSource()).getLogicalProperties().isPresent() && + ((GroupReference) node.getSource()).getLogicalProperties().get().isAtMostSingleRow()); + } + + @Override + public Pattern getPattern() + { + return PATTERN; + } + + @Override + public Result apply(TopNNode node, Captures captures, Context context) + { + return Result.ofPlanNode(node.getSource()); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/SingleDistinctAggregationToGroupBy.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/SingleDistinctAggregationToGroupBy.java index 3fcbf6d419db5..4a9c68b0cd59d 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/SingleDistinctAggregationToGroupBy.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/SingleDistinctAggregationToGroupBy.java @@ -33,10 +33,10 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static com.facebook.presto.spi.plan.AggregationNode.Aggregation.removeDistinct; import static com.facebook.presto.spi.plan.AggregationNode.Step.SINGLE; import static com.facebook.presto.spi.plan.AggregationNode.singleGroupingSet; import static com.facebook.presto.sql.planner.plan.Patterns.aggregation; -import static com.google.common.base.Preconditions.checkArgument; import static java.util.Collections.emptyList; /** @@ -153,16 +153,4 @@ public Result apply(AggregationNode aggregation, Captures captures, Context cont aggregation.getHashVariable(), aggregation.getGroupIdVariable())); } - - private static AggregationNode.Aggregation removeDistinct(AggregationNode.Aggregation aggregation) - { - checkArgument(aggregation.isDistinct(), "Expected aggregation to have DISTINCT input"); - - return new AggregationNode.Aggregation( - aggregation.getCall(), - aggregation.getFilter(), - aggregation.getOrderBy(), - false, - aggregation.getMask()); - } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PruneUnreferencedOutputs.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PruneUnreferencedOutputs.java index 5003f70bda575..b5d5cc39b2644 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PruneUnreferencedOutputs.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PruneUnreferencedOutputs.java @@ -472,6 +472,7 @@ public PlanNode visitTableScan(TableScanNode node, RewriteContext conte node.getTable(), node.getOutputVariables(), newAssignments.build(), + node.getTableConstraints(), node.getCurrentConstraint(), node.getEnforcedConstraint()); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/AssignUniqueId.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/AssignUniqueId.java index 33ac080be5b20..5b7925795de9d 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/AssignUniqueId.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/AssignUniqueId.java @@ -14,6 +14,8 @@ package com.facebook.presto.sql.planner.plan; import com.facebook.presto.spi.SourceLocation; +import com.facebook.presto.spi.plan.LogicalProperties; +import com.facebook.presto.spi.plan.LogicalPropertiesProvider; import com.facebook.presto.spi.plan.PlanNode; import com.facebook.presto.spi.plan.PlanNodeId; import com.facebook.presto.spi.relation.VariableReferenceExpression; @@ -73,6 +75,13 @@ public VariableReferenceExpression getIdVariable() return idVariable; } + @Override + public LogicalProperties computeLogicalProperties(LogicalPropertiesProvider logicalPropertiesProvider) + { + requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider cannot be null."); + return logicalPropertiesProvider.getAssignUniqueIdProperties(this); + } + @Override public R accept(InternalPlanVisitor visitor, C context) { diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/JoinNode.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/JoinNode.java index b118cd7587be3..b80082849ab9c 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/JoinNode.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/JoinNode.java @@ -15,6 +15,8 @@ import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.SourceLocation; +import com.facebook.presto.spi.plan.LogicalProperties; +import com.facebook.presto.spi.plan.LogicalPropertiesProvider; import com.facebook.presto.spi.plan.PlanNode; import com.facebook.presto.spi.plan.PlanNodeId; import com.facebook.presto.spi.relation.RowExpression; @@ -294,6 +296,13 @@ public List getSources() return ImmutableList.of(left, right); } + @Override + public LogicalProperties computeLogicalProperties(LogicalPropertiesProvider logicalPropertiesProvider) + { + requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider cannot be null."); + return logicalPropertiesProvider.getJoinProperties(this); + } + @Override @JsonProperty public List getOutputVariables() diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/SemiJoinNode.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/SemiJoinNode.java index 3e4992f73b0e7..c43d909ef399e 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/SemiJoinNode.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/SemiJoinNode.java @@ -14,6 +14,8 @@ package com.facebook.presto.sql.planner.plan; import com.facebook.presto.spi.SourceLocation; +import com.facebook.presto.spi.plan.LogicalProperties; +import com.facebook.presto.spi.plan.LogicalPropertiesProvider; import com.facebook.presto.spi.plan.PlanNode; import com.facebook.presto.spi.plan.PlanNodeId; import com.facebook.presto.spi.relation.VariableReferenceExpression; @@ -185,6 +187,13 @@ public PlanNode replaceChildren(List newChildren) dynamicFilters); } + @Override + public LogicalProperties computeLogicalProperties(LogicalPropertiesProvider logicalPropertiesProvider) + { + requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider cannot be null."); + return logicalPropertiesProvider.getSemiJoinProperties(this); + } + public SemiJoinNode withDistributionType(DistributionType distributionType) { return new SemiJoinNode( diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/SortNode.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/SortNode.java index 8a5180dd0add1..82e4c623706e0 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/SortNode.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/SortNode.java @@ -14,6 +14,8 @@ package com.facebook.presto.sql.planner.plan; import com.facebook.presto.spi.SourceLocation; +import com.facebook.presto.spi.plan.LogicalProperties; +import com.facebook.presto.spi.plan.LogicalPropertiesProvider; import com.facebook.presto.spi.plan.OrderingScheme; import com.facebook.presto.spi.plan.PlanNode; import com.facebook.presto.spi.plan.PlanNodeId; @@ -65,6 +67,13 @@ public PlanNode getSource() return source; } + @Override + public LogicalProperties computeLogicalProperties(LogicalPropertiesProvider logicalPropertiesProvider) + { + requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider cannot be null."); + return logicalPropertiesProvider.getSortProperties(this); + } + @Override public List getOutputVariables() { diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/planPrinter/PlanPrinter.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/planPrinter/PlanPrinter.java index b48433ce7f984..08a1aaf96972d 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/planPrinter/PlanPrinter.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/planPrinter/PlanPrinter.java @@ -938,6 +938,10 @@ private void printTableScanInfo(NodeRepresentation nodeOutput, TableScanNode nod } } + if (!node.getTableConstraints().isEmpty()) { + nodeOutput.appendDetailsLine("Table Constraints: %s", node.getTableConstraints()); + } + TupleDomain predicate = node.getCurrentConstraint(); if (predicate == null) { // This happens when printing the plan fragment on worker for debug purpose diff --git a/presto-main/src/main/java/com/facebook/presto/sql/relational/FunctionResolution.java b/presto-main/src/main/java/com/facebook/presto/sql/relational/FunctionResolution.java index 19b315b51d1d6..5dac40eef2198 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/relational/FunctionResolution.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/relational/FunctionResolution.java @@ -333,4 +333,9 @@ public FunctionHandle approximateSetFunction(Type valueType) { return functionAndTypeManager.lookupFunction("approx_set", fromTypes(valueType)); } + + public boolean isEqualFunction(FunctionHandle functionHandle) + { + return functionAndTypeManager.getFunctionMetadata(functionHandle).getOperatorType().map(EQUAL::equals).orElse(false); + } } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java b/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java index f1f601057f0a6..0254c78a2cdd2 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java @@ -142,6 +142,7 @@ public void testDefaults() .setFilterAndProjectMinOutputPageSize(new DataSize(500, KILOBYTE)) .setFilterAndProjectMinOutputPageRowCount(256) .setUseMarkDistinct(true) + .setExploitConstraints(false) .setPreferPartialAggregation(true) .setPartialAggregationStrategy(PartialAggregationStrategy.ALWAYS) .setPartialAggregationByteReductionThreshold(0.5) @@ -293,6 +294,7 @@ public void testExplicitPropertyMappings() .put("arrayagg.implementation", "LEGACY") .put("multimapagg.implementation", "LEGACY") .put("optimizer.use-mark-distinct", "false") + .put("optimizer.exploit-constraints", "true") .put("optimizer.prefer-partial-aggregation", "false") .put("optimizer.partial-aggregation-strategy", "automatic") .put("optimizer.partial-aggregation-byte-reduction-threshold", "0.8") @@ -434,6 +436,7 @@ public void testExplicitPropertyMappings() .setFilterAndProjectMinOutputPageSize(new DataSize(1, MEGABYTE)) .setFilterAndProjectMinOutputPageRowCount(2048) .setUseMarkDistinct(false) + .setExploitConstraints(true) .setPreferPartialAggregation(false) .setPartialAggregationStrategy(PartialAggregationStrategy.AUTOMATIC) .setPartialAggregationByteReductionThreshold(0.8) diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestCanonicalPlanGenerator.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestCanonicalPlanGenerator.java index 2ee351f041da1..d9d2fc457bc54 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestCanonicalPlanGenerator.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestCanonicalPlanGenerator.java @@ -262,7 +262,7 @@ public void testCanonicalTableScanNodeField() .filter(f -> !f.isSynthetic()) .map(Field::getName) .collect(toImmutableSet()), - ImmutableSet.of("table", "assignments", "outputVariables", "currentConstraint", "enforcedConstraint")); + ImmutableSet.of("table", "assignments", "outputVariables", "currentConstraint", "enforcedConstraint", "tableConstraints")); assertEquals( Arrays.stream(CanonicalTableScanNode.class.getDeclaredFields()) .filter(f -> !f.isSynthetic()) diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestLogicalPlanner.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestLogicalPlanner.java index eb331310a555d..857dac83fb37e 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestLogicalPlanner.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestLogicalPlanner.java @@ -17,8 +17,10 @@ import com.facebook.presto.spi.plan.AggregationNode; import com.facebook.presto.spi.plan.DistinctLimitNode; import com.facebook.presto.spi.plan.FilterNode; +import com.facebook.presto.spi.plan.LimitNode; import com.facebook.presto.spi.plan.PlanNode; import com.facebook.presto.spi.plan.TableScanNode; +import com.facebook.presto.spi.plan.TopNNode; import com.facebook.presto.spi.plan.ValuesNode; import com.facebook.presto.spi.relation.VariableReferenceExpression; import com.facebook.presto.sql.analyzer.FeaturesConfig.JoinDistributionType; @@ -35,6 +37,7 @@ import com.facebook.presto.sql.planner.plan.JoinNode; import com.facebook.presto.sql.planner.plan.LateralJoinNode; import com.facebook.presto.sql.planner.plan.SemiJoinNode; +import com.facebook.presto.sql.planner.plan.SortNode; import com.facebook.presto.sql.planner.plan.StatisticsWriterNode; import com.facebook.presto.sql.tree.LongLiteral; import com.facebook.presto.tests.QueryTemplate; @@ -50,6 +53,7 @@ import static com.facebook.presto.SystemSessionProperties.DISTRIBUTED_SORT; import static com.facebook.presto.SystemSessionProperties.ENFORCE_FIXED_DISTRIBUTION_FOR_OUTPUT_OPERATOR; +import static com.facebook.presto.SystemSessionProperties.EXPLOIT_CONSTRAINTS; import static com.facebook.presto.SystemSessionProperties.FORCE_SINGLE_NODE_OUTPUT; import static com.facebook.presto.SystemSessionProperties.JOIN_DISTRIBUTION_TYPE; import static com.facebook.presto.SystemSessionProperties.JOIN_REORDERING_STRATEGY; @@ -67,6 +71,7 @@ import static com.facebook.presto.spi.plan.AggregationNode.Step.PARTIAL; import static com.facebook.presto.spi.plan.AggregationNode.Step.SINGLE; import static com.facebook.presto.sql.analyzer.FeaturesConfig.JoinReorderingStrategy.ELIMINATE_CROSS_JOINS; +import static com.facebook.presto.sql.planner.LogicalPlanner.Stage.OPTIMIZED; import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.aggregation; import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.any; import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.anyNot; @@ -600,7 +605,7 @@ public void testJoinOutputPruning() private void assertPlanContainsNoApplyOrAnyJoin(String sql) { assertFalse( - searchFrom(plan(sql, LogicalPlanner.Stage.OPTIMIZED).getRoot()) + searchFrom(plan(sql, OPTIMIZED).getRoot()) .where(isInstanceOfAny(ApplyNode.class, JoinNode.class, IndexJoinNode.class, SemiJoinNode.class, LateralJoinNode.class)) .matches(), "Unexpected node for query: " + sql); @@ -611,7 +616,7 @@ public void testCorrelatedSubqueries() { assertPlan( "SELECT orderkey FROM orders WHERE 3 = (SELECT orderkey)", - LogicalPlanner.Stage.OPTIMIZED, + OPTIMIZED, any( filter( "X = BIGINT '3'", @@ -772,7 +777,7 @@ public void testDoubleNestedCorrelatedSubqueries() assertPlan( "SELECT orderkey FROM orders o " + "WHERE 3 IN (SELECT o.custkey FROM lineitem l WHERE (SELECT l.orderkey = o.orderkey))", - LogicalPlanner.Stage.OPTIMIZED, + OPTIMIZED, anyTree( filter("OUTER_FILTER", apply(ImmutableList.of("C", "O"), @@ -1396,4 +1401,97 @@ private Session noJoinReordering() .setSystemProperty(JOIN_DISTRIBUTION_TYPE, JoinDistributionType.PARTITIONED.name()) .build(); } + + public void testRedundantLimitNodeRemoval() + { + String query = "SELECT count(*) FROM orders LIMIT 10"; + assertFalse( + searchFrom(plan(query, OPTIMIZED).getRoot()) + .where(LimitNode.class::isInstance) + .matches(), + format("Unexpected limit node for query: '%s'", query)); + + assertPlan( + "SELECT orderkey, count(*) FROM orders GROUP BY orderkey LIMIT 10", + output( + limit(10, + anyTree( + tableScan("orders"))))); + + assertPlan( + "SELECT * FROM (VALUES 1,2,3,4,5,6) AS t1 LIMIT 10", + output( + values(ImmutableList.of("x")))); + } + + @Test + public void testRemoveSingleRowSort() + { + Session exploitConstraints = Session.builder(this.getQueryRunner().getDefaultSession()) + .setSystemProperty(EXPLOIT_CONSTRAINTS, Boolean.toString(true)) + .build(); + + String query = "SELECT count(*) FROM orders ORDER BY 1"; + assertFalse( + searchFrom(plan(query, OPTIMIZED, exploitConstraints).getRoot()) + .where(isInstanceOfAny(SortNode.class)) + .matches(), + format("Unexpected sort node for query: '%s'", query)); + + assertPlan( + "SELECT orderkey, count(*) FROM orders GROUP BY orderkey ORDER BY 1", + anyTree( + node(SortNode.class, + anyTree( + tableScan("orders"))))); + } + + @Test + public void testRedundantTopNNodeRemoval() + { + Session exploitConstraints = Session.builder(this.getQueryRunner().getDefaultSession()) + .setSystemProperty(EXPLOIT_CONSTRAINTS, Boolean.toString(true)) + .build(); + + String query = "SELECT count(*) FROM orders ORDER BY 1 LIMIT 10"; + assertFalse( + searchFrom(plan(query, OPTIMIZED, exploitConstraints).getRoot()) + .where(isInstanceOfAny(TopNNode.class, SortNode.class)) + .matches(), + format("Unexpected TopN node for query: '%s'", query)); + + assertPlan( + "SELECT orderkey, count(*) FROM orders GROUP BY orderkey ORDER BY 1 LIMIT 10", + output( + node(TopNNode.class, + anyTree( + tableScan("orders"))))); + + assertPlan( + "SELECT orderkey, count(*) FROM orders GROUP BY orderkey ORDER BY 1 LIMIT 0", + output( + node(ValuesNode.class))); + } + + @Test + public void testRedundantDistinctLimitNodeRemoval() + { + Session exploitConstraints = Session.builder(this.getQueryRunner().getDefaultSession()) + .setSystemProperty(EXPLOIT_CONSTRAINTS, Boolean.toString(true)) + .build(); + + String query = "SELECT distinct(c) FROM (SELECT count(*) as c FROM orders) LIMIT 10"; + assertFalse( + searchFrom(plan(query, OPTIMIZED, exploitConstraints).getRoot()) + .where(isInstanceOfAny(DistinctLimitNode.class)) + .matches(), + format("Unexpected DistinctLimit node for query: '%s'", query)); + + assertPlan( + "SELECT distinct(c) FROM (SELECT count(*) as c FROM orders GROUP BY orderkey) LIMIT 10", + output( + node(DistinctLimitNode.class, + anyTree( + tableScan("orders"))))); + } } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestPushDownDereferences.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestPushDownDereferences.java index 3c9bfbbe552eb..964d06df4c3b7 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestPushDownDereferences.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestPushDownDereferences.java @@ -167,7 +167,7 @@ public void testLimit() public void testSort() { ImmutableList orderBy = ImmutableList.of(sort("b_x", ASCENDING, LAST)); - assertPlan("WITH t(msg) AS ( SELECT * FROM (VALUES ROW(CAST(ROW(1, 2.0) AS ROW(x BIGINT, y DOUBLE))))) " + + assertPlan("WITH t(msg) AS ( SELECT * FROM (VALUES ROW(CAST(ROW(1, 2.0) AS ROW(x BIGINT, y DOUBLE))), ROW(CAST(ROW(1, 2.0) AS ROW(x BIGINT, y DOUBLE))))) " + "SELECT a.msg.x FROM t a JOIN t b ON a.msg.y = b.msg.y WHERE a.msg.x < bigint '10' ORDER BY b.msg.x", output(ImmutableList.of("expr"), project(ImmutableMap.of("expr", expression("a_x")), diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestTableConstraintsConnectorFactory.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestTableConstraintsConnectorFactory.java new file mode 100644 index 0000000000000..1d296072a718a --- /dev/null +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestTableConstraintsConnectorFactory.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 com.facebook.presto.sql.planner; + +import com.facebook.presto.spi.NodeManager; +import com.facebook.presto.spi.connector.Connector; +import com.facebook.presto.spi.connector.ConnectorContext; +import com.facebook.presto.spi.connector.ConnectorMetadata; +import com.facebook.presto.spi.connector.ConnectorNodePartitioningProvider; +import com.facebook.presto.spi.connector.ConnectorRecordSetProvider; +import com.facebook.presto.spi.connector.ConnectorSplitManager; +import com.facebook.presto.spi.connector.ConnectorTransactionHandle; +import com.facebook.presto.spi.transaction.IsolationLevel; +import com.facebook.presto.tpch.ColumnNaming; +import com.facebook.presto.tpch.TpchConnectorFactory; +import com.facebook.presto.tpch.TpchNodePartitioningProvider; +import com.facebook.presto.tpch.TpchRecordSetProvider; +import com.facebook.presto.tpch.TpchSplitManager; +import com.facebook.presto.tpch.TpchTransactionHandle; + +import java.util.Map; + +public class TestTableConstraintsConnectorFactory + extends TpchConnectorFactory +{ + public TestTableConstraintsConnectorFactory(int defaultSplitsPerNode) + { + super(defaultSplitsPerNode); + } + + @Override + public Connector create(String catalogName, Map properties, ConnectorContext context) + { + int splitsPerNode = super.getSplitsPerNode(properties); + ColumnNaming columnNaming = ColumnNaming.valueOf(properties.getOrDefault(TPCH_COLUMN_NAMING_PROPERTY, ColumnNaming.SIMPLIFIED.name()).toUpperCase()); + NodeManager nodeManager = context.getNodeManager(); + + return new Connector() + { + @Override + public ConnectorTransactionHandle beginTransaction(IsolationLevel isolationLevel, boolean readOnly) + { + return TpchTransactionHandle.INSTANCE; + } + + @Override + public ConnectorMetadata getMetadata(ConnectorTransactionHandle transaction) + { + return new TestTableConstraintsMetadata(catalogName, columnNaming, isPredicatePushdownEnabled(), isPartitioningEnabled(properties)); + } + + @Override + public ConnectorSplitManager getSplitManager() + { + return new TpchSplitManager(nodeManager, splitsPerNode); + } + + @Override + public ConnectorRecordSetProvider getRecordSetProvider() + { + return new TpchRecordSetProvider(); + } + + @Override + public ConnectorNodePartitioningProvider getNodePartitioningProvider() + { + return new TpchNodePartitioningProvider(nodeManager, splitsPerNode); + } + }; + } +} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestTableConstraintsMetadata.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestTableConstraintsMetadata.java new file mode 100644 index 0000000000000..422fa02a9f233 --- /dev/null +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestTableConstraintsMetadata.java @@ -0,0 +1,196 @@ +/* + * 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 com.facebook.presto.sql.planner; + +import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.ColumnMetadata; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.ConnectorTableHandle; +import com.facebook.presto.spi.ConnectorTableMetadata; +import com.facebook.presto.spi.SchemaTableName; +import com.facebook.presto.spi.constraints.PrimaryKeyConstraint; +import com.facebook.presto.spi.constraints.TableConstraint; +import com.facebook.presto.spi.constraints.UniqueConstraint; +import com.facebook.presto.tpch.ColumnNaming; +import com.facebook.presto.tpch.TpchColumnHandle; +import com.facebook.presto.tpch.TpchMetadata; +import com.facebook.presto.tpch.TpchTableHandle; +import com.google.common.collect.ImmutableList; +import io.airlift.tpch.TpchColumn; +import io.airlift.tpch.TpchEntity; +import io.airlift.tpch.TpchTable; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static com.facebook.presto.common.type.BigintType.BIGINT; +import static java.util.Collections.emptyMap; + +public class TestTableConstraintsMetadata + extends TpchMetadata +{ + public TestTableConstraintsMetadata(String connectorId, ColumnNaming columnNaming, boolean predicatePushdownEnabled, boolean partitioningEnabled) + { + super(connectorId, columnNaming, predicatePushdownEnabled, partitioningEnabled); + } + + private static ConnectorTableMetadata getTableConstraintsMetadata(String schemaName, TpchTable tpchTable, ColumnNaming columnNaming) + { + ImmutableList.Builder columns = ImmutableList.builder(); + Map nameToHandleMap = new HashMap<>(); //SCAFFOLDING + for (TpchColumn column : tpchTable.getColumns()) { + columns.add(new ColumnMetadata(columnNaming.getName(column), getPrestoType(column), false, null, null, false, emptyMap())); + nameToHandleMap.put(columnNaming.getName(column), new TpchColumnHandle(columnNaming.getName(column), getPrestoType(column))); //SCAFFOLDING + } + columns.add(new ColumnMetadata(ROW_NUMBER_COLUMN_NAME, BIGINT, null, true)); + SchemaTableName tableName = new SchemaTableName(schemaName, tpchTable.getTableName()); + List> constraints = getTableConstraints(tpchTable.getTableName()); + return new ConnectorTableMetadata(tableName, columns.build(), emptyMap(), Optional.empty(), ConnectorTableMetadata.rebaseTableConstraints(constraints, nameToHandleMap)); + } + + private static List> getTableConstraints(String tableName) + { + //Primary key constraints + List> constraints = new ArrayList<>(); + TpchTableConstraints tpchProperties = new TpchTableConstraints(); + for (Set pkcols : tpchProperties.lookupPK(tableName)) { + PrimaryKeyConstraint pk = new PrimaryKeyConstraint(tableName + "pk", pkcols, false, true); + constraints.add(pk); + } + //Unique constraints + for (Set pkcols : tpchProperties.lookupUKs(tableName)) { + UniqueConstraint pk = new UniqueConstraint(tableName + "pk", pkcols, false, true); + constraints.add(pk); + } + return constraints; + } + + @Override + public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle tableHandle) + { + TpchTableHandle tpchTableHandle = (TpchTableHandle) tableHandle; + TpchTable tpchTable = TpchTable.getTable(tpchTableHandle.getTableName()); + String schemaName = TpchMetadata.scaleFactorSchemaName(tpchTableHandle.getScaleFactor()); + return getTableConstraintsMetadata(schemaName, tpchTable, getColumnNaming()); + } + + private static class TpchTableConstraints + { + private final String customerTableName = "customer"; + private final String lineitemTableName = "lineitem"; + private final String ordersTableName = "orders"; + private final String partTableName = "part"; + private final String suppliersTableName = "supplier"; + private final String partsuppTableName = "partsupp"; + private final String nationTableName = "nation"; + private final String regionTableName = "region"; + + //primary keys + private final Set customerPK = new HashSet<>(); + private final Set lineItemPK = new HashSet<>(); + private final Set ordersPK = new HashSet<>(); + private final Set partPK = new HashSet<>(); + private final Set suppliersPK = new HashSet<>(); + private final Set partsuppPK = new HashSet<>(); + private final Set nationPK = new HashSet<>(); + private final Set regionPK = new HashSet<>(); + private final Map>> tpchPKLookup = new HashMap>>(); + + //unique keys + private final Set ordersKey1 = new HashSet<>(); + private final Set ordersKey2 = new HashSet<>(); + private final Map>> tpchUKLookup = new HashMap>>(); + + public TpchTableConstraints() + { + //customer + ArrayList> customerKeys = new ArrayList<>(); + customerPK.add("custkey"); + customerKeys.add(customerPK); + tpchPKLookup.put(customerTableName, customerKeys); + + //orders + ArrayList> ordersKeys = new ArrayList<>(); + ordersPK.add("orderkey"); + ordersKeys.add(ordersPK); + tpchPKLookup.put(ordersTableName, ordersKeys); + + // add supperfulous unique key + ArrayList> ordersUniqueKeys = new ArrayList<>(); + ordersKey1.add("orderkey"); + ordersKey1.add("custkey"); + ordersUniqueKeys.add(ordersKey1); + + // add supperfulous unique key + ordersKey2.add("orderkey"); + ordersKey2.add("orderdate"); + ordersUniqueKeys.add(ordersKey2); + tpchUKLookup.put(ordersTableName, ordersUniqueKeys); + + //lineitem + ArrayList> lineitemKeys = new ArrayList<>(); + lineItemPK.add("linenumber"); + lineItemPK.add("orderkey"); + lineitemKeys.add(lineItemPK); + tpchPKLookup.put(lineitemTableName, lineitemKeys); + + //part + ArrayList> partKeys = new ArrayList<>(); + partPK.add("partkey"); + partKeys.add(partPK); + tpchPKLookup.put(partTableName, partKeys); + + //suppliers + ArrayList> suppliersKeys = new ArrayList<>(); + suppliersPK.add("suppkey"); + suppliersKeys.add(suppliersPK); + tpchPKLookup.put(suppliersTableName, suppliersKeys); + + //partsupp + ArrayList> partsuppKeys = new ArrayList<>(); + partsuppPK.add("partkey"); + partsuppPK.add("suppkey"); + partsuppKeys.add(partsuppPK); + tpchPKLookup.put(partsuppTableName, partsuppKeys); + + //nation + ArrayList> nationKeys = new ArrayList<>(); + nationPK.add("nationkey"); + nationKeys.add(nationPK); + tpchPKLookup.put(nationTableName, nationKeys); + + //region + ArrayList> regionKeys = new ArrayList<>(); + regionPK.add("regionkey"); + regionKeys.add(regionPK); + tpchPKLookup.put(regionTableName, regionKeys); + } + + public List> lookupPK(String tableName) + { + return tpchPKLookup.containsKey(tableName) ? tpchPKLookup.get(tableName) : Collections.emptyList(); + } + + public List> lookupUKs(String tableName) + { + return tpchUKLookup.containsKey(tableName) ? tpchUKLookup.get(tableName) : Collections.emptyList(); + } + } +} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/BasePlanTest.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/BasePlanTest.java index 8e32a5e85649d..10de360cc8ce8 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/BasePlanTest.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/BasePlanTest.java @@ -198,6 +198,15 @@ protected void assertPlanWithSession(@Language("SQL") String sql, Session sessio }); } + protected void assertPlanValidatorWithSession(@Language("SQL") String sql, Session session, boolean forceSingleNode, Consumer planValidator) + { + queryRunner.inTransaction(session, transactionSession -> { + Plan actualPlan = queryRunner.createPlan(transactionSession, sql, LogicalPlanner.Stage.OPTIMIZED_AND_VALIDATED, forceSingleNode, WarningCollector.NOOP); + planValidator.accept(actualPlan); + return null; + }); + } + protected Plan plan(String sql) { return plan(sql, LogicalPlanner.Stage.OPTIMIZED_AND_VALIDATED); @@ -218,6 +227,21 @@ protected Plan plan(String sql, LogicalPlanner.Stage stage, boolean forceSingleN } } + protected Plan plan(String sql, LogicalPlanner.Stage stage, Session session) + { + return plan(sql, stage, true, session); + } + + protected Plan plan(String sql, LogicalPlanner.Stage stage, boolean forceSingleNode, Session session) + { + try { + return queryRunner.inTransaction(session, transactionSession -> queryRunner.createPlan(transactionSession, sql, stage, forceSingleNode, WarningCollector.NOOP)); + } + catch (RuntimeException e) { + throw new AssertionError("Planning failed for SQL: " + sql, e); + } + } + protected SubPlan subplan(String sql, LogicalPlanner.Stage stage, boolean forceSingleNode) { return subplan(sql, stage, forceSingleNode, getQueryRunner().getDefaultSession()); diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/OptimizerAssert.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/OptimizerAssert.java index 811258727d9b9..8506739bba4af 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/OptimizerAssert.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/OptimizerAssert.java @@ -1,17 +1,3 @@ -/* - * 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. - */ - /* * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,18 +21,32 @@ import com.facebook.presto.spi.WarningCollector; import com.facebook.presto.spi.plan.PlanNode; import com.facebook.presto.spi.plan.PlanNodeIdAllocator; +import com.facebook.presto.sql.planner.LogicalPlanner; import com.facebook.presto.sql.planner.Plan; import com.facebook.presto.sql.planner.PlanVariableAllocator; +import com.facebook.presto.sql.planner.RuleStatsRecorder; import com.facebook.presto.sql.planner.TypeProvider; +import com.facebook.presto.sql.planner.iterative.IterativeOptimizer; +import com.facebook.presto.sql.planner.iterative.rule.RemoveRedundantIdentityProjections; +import com.facebook.presto.sql.planner.iterative.rule.SimplifyRowExpressions; +import com.facebook.presto.sql.planner.iterative.rule.TransformUncorrelatedInPredicateSubqueryToSemiJoin; +import com.facebook.presto.sql.planner.iterative.rule.TranslateExpressions; import com.facebook.presto.sql.planner.iterative.rule.test.PlanBuilder; import com.facebook.presto.sql.planner.iterative.rule.test.RuleAssert.TestingStatsCalculator; import com.facebook.presto.sql.planner.optimizations.PlanOptimizer; +import com.facebook.presto.sql.planner.optimizations.PruneUnreferencedOutputs; +import com.facebook.presto.sql.planner.optimizations.UnaliasSymbolReferences; +import com.facebook.presto.testing.LocalQueryRunner; import com.facebook.presto.transaction.TransactionManager; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import java.util.List; +import java.util.function.Consumer; import java.util.function.Function; import static com.facebook.presto.sql.planner.assertions.PlanAssert.assertPlan; +import static com.facebook.presto.sql.planner.assertions.PlanAssert.assertPlanDoesNotMatch; import static com.facebook.presto.transaction.TransactionBuilder.transaction; import static com.google.common.base.Preconditions.checkState; import static java.util.Objects.requireNonNull; @@ -61,11 +61,12 @@ public class OptimizerAssert private final PlanNodeIdAllocator idAllocator = new PlanNodeIdAllocator(); private final TransactionManager transactionManager; private final AccessControl accessControl; + private final LocalQueryRunner queryRunner; private TypeProvider types; private PlanNode plan; - public OptimizerAssert(Metadata metadata, StatsCalculator statsCalculator, Session session, PlanOptimizer optimizer, TransactionManager transactionManager, AccessControl accessControl) + public OptimizerAssert(Metadata metadata, LocalQueryRunner queryRunner, StatsCalculator statsCalculator, Session session, PlanOptimizer optimizer, TransactionManager transactionManager, AccessControl accessControl) { this.metadata = requireNonNull(metadata, "metadata is null"); this.statsCalculator = new TestingStatsCalculator(requireNonNull(statsCalculator, "statsCalculator is null")); @@ -73,6 +74,7 @@ public OptimizerAssert(Metadata metadata, StatsCalculator statsCalculator, Sessi this.optimizer = requireNonNull(optimizer, "optimizer is null"); this.transactionManager = requireNonNull(transactionManager, "transactionManager is null"); this.accessControl = requireNonNull(accessControl, "access control is null"); + this.queryRunner = requireNonNull(queryRunner, "queryRunner is null"); } public OptimizerAssert on(Function planProvider) @@ -85,7 +87,39 @@ public OptimizerAssert on(Function planProvider) return this; } + public OptimizerAssert on(String sql) + { + checkState(plan == null, "plan has already been set"); + + //get an initial plan and apply a minimal set of optimizers in preparation foor applying the specific rules to be tested + Plan result = queryRunner.inTransaction(session -> queryRunner.createPlan(session, sql, getMinimalOptimizers(), LogicalPlanner.Stage.OPTIMIZED, WarningCollector.NOOP)); + plan = result.getRoot(); + types = result.getTypes(); + return this; + } + public void matches(PlanMatchPattern pattern) + { + inTransaction(session -> { + assertPlan(session, metadata, statsCalculator, applyRules(), pattern); + return null; + }); + } + + public void doesNotMatch(PlanMatchPattern pattern) + { + inTransaction(session -> { + assertPlanDoesNotMatch(session, metadata, statsCalculator, applyRules(), pattern); + return null; + }); + } + + public void validates(Consumer planValidator) + { + planValidator.accept(applyRules()); + } + + private Plan applyRules() { PlanNode actual = optimizer.optimize(plan, session, types, new PlanVariableAllocator(), idAllocator, WarningCollector.NOOP); @@ -98,11 +132,34 @@ public void matches(PlanMatchPattern pattern) plan.getOutputVariables(), actual.getOutputVariables())); } + return new Plan(actual, types, StatsAndCosts.empty()); + } - inTransaction(session -> { - assertPlan(session, metadata, statsCalculator, new Plan(actual, types, StatsAndCosts.empty()), pattern); - return null; - }); + private List getMinimalOptimizers() + { + return ImmutableList.of( + new UnaliasSymbolReferences(queryRunner.getMetadata().getFunctionAndTypeManager()), + new PruneUnreferencedOutputs(), + new IterativeOptimizer( + new RuleStatsRecorder(), + queryRunner.getStatsCalculator(), + queryRunner.getCostCalculator(), + ImmutableSet.of(new TransformUncorrelatedInPredicateSubqueryToSemiJoin(), new RemoveRedundantIdentityProjections())), + getExpressionTranslator(), + new IterativeOptimizer( + new RuleStatsRecorder(), + queryRunner.getStatsCalculator(), + queryRunner.getCostCalculator(), + new SimplifyRowExpressions(metadata).rules())); + } + + private PlanOptimizer getExpressionTranslator() + { + return new IterativeOptimizer( + new RuleStatsRecorder(), + queryRunner.getStatsCalculator(), + queryRunner.getCostCalculator(), + ImmutableSet.copyOf(new TranslateExpressions(metadata, queryRunner.getSqlParser()).rules())); } private void inTransaction(Function transactionSessionConsumer) diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanAssert.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanAssert.java index 6db995cb39364..79aa47c1e187f 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanAssert.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanAssert.java @@ -40,6 +40,12 @@ public static void assertPlan(Session session, Metadata metadata, StatsCalculato assertPlan(session, metadata, statsProvider, actual, noLookup(), pattern, Function.identity()); } + public static void assertPlanDoesNotMatch(Session session, Metadata metadata, StatsCalculator statsCalculator, Plan actual, PlanMatchPattern pattern) + { + StatsProvider statsProvider = new CachingStatsProvider(statsCalculator, session, actual.getTypes()); + assertPlanDoesNotMatch(session, metadata, statsProvider, actual, noLookup(), pattern, Function.identity()); + } + public static void assertPlan(Session session, Metadata metadata, StatsProvider statsProvider, Plan actual, Lookup lookup, PlanMatchPattern pattern, Function planSanitizer) { MatchResult matches = actual.getRoot().accept(new PlanMatchingVisitor(session, metadata, statsProvider, lookup), pattern); @@ -53,4 +59,18 @@ public static void assertPlan(Session session, Metadata metadata, StatsProvider resolvedFormattedPlan)); } } + + public static void assertPlanDoesNotMatch(Session session, Metadata metadata, StatsProvider statsProvider, Plan actual, Lookup lookup, PlanMatchPattern pattern, Function planSanitizer) + { + MatchResult matches = actual.getRoot().accept(new PlanMatchingVisitor(session, metadata, statsProvider, lookup), pattern); + // TODO (Issue #13231) add back printing unresolved plan once we have no need to translate OriginalExpression to RowExpression + if (matches.isMatch()) { + PlanNode resolvedPlan = resolveGroupReferences(actual.getRoot(), lookup); + String resolvedFormattedPlan = textLogicalPlan(planSanitizer.apply(resolvedPlan), actual.getTypes(), metadata.getFunctionAndTypeManager(), StatsAndCosts.empty(), session, 0); + throw new AssertionError(format( + "Plan unexpectedly matches the pattern, pattern is [\n\n%s\n] and plan is [\n\n%s\n]", + pattern, + resolvedFormattedPlan)); + } + } } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestLogicalPropertyPropagation.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestLogicalPropertyPropagation.java new file mode 100644 index 0000000000000..201f7164b0f66 --- /dev/null +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestLogicalPropertyPropagation.java @@ -0,0 +1,2253 @@ +/* + * 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 com.facebook.presto.sql.planner.iterative.rule; + +import com.facebook.presto.common.predicate.TupleDomain; +import com.facebook.presto.matching.Captures; +import com.facebook.presto.matching.Pattern; +import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.ConnectorId; +import com.facebook.presto.spi.TableHandle; +import com.facebook.presto.spi.constraints.PrimaryKeyConstraint; +import com.facebook.presto.spi.constraints.TableConstraint; +import com.facebook.presto.spi.constraints.UniqueConstraint; +import com.facebook.presto.spi.plan.Assignments; +import com.facebook.presto.spi.plan.FilterNode; +import com.facebook.presto.spi.plan.LimitNode; +import com.facebook.presto.spi.plan.LogicalProperties; +import com.facebook.presto.spi.plan.PlanNode; +import com.facebook.presto.spi.plan.TableScanNode; +import com.facebook.presto.spi.plan.ValuesNode; +import com.facebook.presto.spi.relation.ConstantExpression; +import com.facebook.presto.spi.relation.VariableReferenceExpression; +import com.facebook.presto.sql.planner.TestTableConstraintsConnectorFactory; +import com.facebook.presto.sql.planner.iterative.Rule; +import com.facebook.presto.sql.planner.iterative.properties.EquivalenceClassProperty; +import com.facebook.presto.sql.planner.iterative.properties.Key; +import com.facebook.presto.sql.planner.iterative.properties.KeyProperty; +import com.facebook.presto.sql.planner.iterative.properties.LogicalPropertiesImpl; +import com.facebook.presto.sql.planner.iterative.properties.LogicalPropertiesProviderImpl; +import com.facebook.presto.sql.planner.iterative.properties.MaxCardProperty; +import com.facebook.presto.sql.planner.iterative.rule.test.BaseRuleTest; +import com.facebook.presto.sql.planner.iterative.rule.test.RuleTester; +import com.facebook.presto.sql.planner.plan.JoinNode; +import com.facebook.presto.sql.relational.FunctionResolution; +import com.facebook.presto.testing.TestingTransactionHandle; +import com.facebook.presto.tpch.TpchColumnHandle; +import com.facebook.presto.tpch.TpchTableHandle; +import com.facebook.presto.tpch.TpchTableLayoutHandle; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableListMultimap; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import io.airlift.slice.Slices; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static com.facebook.presto.common.type.BigintType.BIGINT; +import static com.facebook.presto.common.type.DoubleType.DOUBLE; +import static com.facebook.presto.common.type.IntegerType.INTEGER; +import static com.facebook.presto.common.type.VarcharType.VARCHAR; +import static com.facebook.presto.common.type.VarcharType.createVarcharType; +import static com.facebook.presto.sql.relational.Expressions.constant; +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Collections.emptyList; + +public class TestLogicalPropertyPropagation + extends BaseRuleTest +{ + private TableHandle customerTableHandle; + private TableHandle ordersTableHandle; + private TableHandle lineitemTableHandle; + + private ColumnHandle customerCustKeyColumn; + private ColumnHandle customerNationKeyColumn; + private ColumnHandle customerCommentColumn; + private ColumnHandle mktSegmentColumn; + private ColumnHandle acctBalColumn; + private ColumnHandle ordersCustKeyColumn; + private ColumnHandle ordersOrderKeyColumn; + private ColumnHandle ordersOrderPriorityColumn; + private ColumnHandle ordersCommentColumn; + private ColumnHandle shipPriorityColumn; + private ColumnHandle lineitemOrderkeyColumn; + private ColumnHandle lineitemLinenumberColumn; + private ColumnHandle lineitemExtendedPriceColumn; + + private VariableReferenceExpression customerCustKeyVariable; + private VariableReferenceExpression customerNationKeyVariable; + private VariableReferenceExpression customerCommentVariable; + private VariableReferenceExpression shipPriorityVariable; + private VariableReferenceExpression mktSegmentVariable; + private VariableReferenceExpression acctBalVariable; + private VariableReferenceExpression ordersCustKeyVariable; + private VariableReferenceExpression ordersOrderKeyVariable; + private VariableReferenceExpression ordersOrderPriorityVariable; + private VariableReferenceExpression ordersCommentVariable; + private VariableReferenceExpression lineitemOrderkeyVariable; + private VariableReferenceExpression lineitemLinenumberVariable; + private VariableReferenceExpression lineitemExtendedPriceVariable; + + private FunctionResolution functionResolution; + private LogicalPropertiesProviderImpl logicalPropertiesProvider; + + @BeforeClass + public final void setUp() + { + tester = new RuleTester(emptyList(), ImmutableMap.of(), Optional.of(1), new TestTableConstraintsConnectorFactory(1)); + ConnectorId connectorId = tester().getCurrentConnectorId(); + functionResolution = new FunctionResolution(tester.getMetadata().getFunctionAndTypeManager()); + logicalPropertiesProvider = new LogicalPropertiesProviderImpl(functionResolution); + + TpchTableHandle customerTpchTableHandle = new TpchTableHandle("customer", 1.0); + TpchTableHandle ordersTpchTableHandle = new TpchTableHandle("orders", 1.0); + TpchTableHandle lineitemTpchTableHandle = new TpchTableHandle("lineitem", 1.0); + + customerTableHandle = new TableHandle( + connectorId, + customerTpchTableHandle, + TestingTransactionHandle.create(), + Optional.of(new TpchTableLayoutHandle(customerTpchTableHandle, TupleDomain.all()))); + + ordersTableHandle = new TableHandle( + connectorId, + ordersTpchTableHandle, + TestingTransactionHandle.create(), + Optional.of(new TpchTableLayoutHandle(ordersTpchTableHandle, TupleDomain.all()))); + + lineitemTableHandle = new TableHandle( + connectorId, + lineitemTpchTableHandle, + TestingTransactionHandle.create(), + Optional.of(new TpchTableLayoutHandle(lineitemTpchTableHandle, TupleDomain.all()))); + + customerCustKeyColumn = new TpchColumnHandle("custkey", BIGINT); + customerCommentColumn = new TpchColumnHandle("comment", VARCHAR); + customerNationKeyColumn = new TpchColumnHandle("nationkey", BIGINT); + mktSegmentColumn = new TpchColumnHandle("mktsegment", VARCHAR); + acctBalColumn = new TpchColumnHandle("acctbal", DOUBLE); + ordersCustKeyColumn = new TpchColumnHandle("custkey", BIGINT); + ordersOrderKeyColumn = new TpchColumnHandle("orderkey", BIGINT); + ordersOrderPriorityColumn = new TpchColumnHandle("orderpriority", BIGINT); + shipPriorityColumn = new TpchColumnHandle("shippriority", INTEGER); + ordersCommentColumn = new TpchColumnHandle("comment", VARCHAR); + lineitemOrderkeyColumn = new TpchColumnHandle("orderkey", BIGINT); + lineitemLinenumberColumn = new TpchColumnHandle("linenumber", BIGINT); + lineitemExtendedPriceColumn = new TpchColumnHandle("extendedprice", DOUBLE); + + customerCustKeyVariable = new VariableReferenceExpression(Optional.empty(), "c_custkey", BIGINT); + customerNationKeyVariable = new VariableReferenceExpression(Optional.empty(), "nationkey", BIGINT); + customerCommentVariable = new VariableReferenceExpression(Optional.empty(), "c_comment", VARCHAR); + mktSegmentVariable = new VariableReferenceExpression(Optional.empty(), "c_mktsegment", VARCHAR); + acctBalVariable = new VariableReferenceExpression(Optional.empty(), "c_acctbal", DOUBLE); + ordersCustKeyVariable = new VariableReferenceExpression(Optional.empty(), "o_custkey", BIGINT); + ordersOrderKeyVariable = new VariableReferenceExpression(Optional.empty(), "o_orderkey", BIGINT); + ordersOrderPriorityVariable = new VariableReferenceExpression(Optional.empty(), "o_orderpriority", VARCHAR); + shipPriorityVariable = new VariableReferenceExpression(Optional.empty(), "o_shippriority", INTEGER); + ordersCommentVariable = new VariableReferenceExpression(Optional.empty(), "o_comment", DOUBLE); + lineitemOrderkeyVariable = new VariableReferenceExpression(Optional.empty(), "l_orderkey", BIGINT); + lineitemLinenumberVariable = new VariableReferenceExpression(Optional.empty(), "l_linenumber", BIGINT); + lineitemExtendedPriceVariable = new VariableReferenceExpression(Optional.empty(), "l_extendedprice", DOUBLE); + } + + @Test + void testValuesNodeLogicalProperties() + { + LogicalProperties expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(1L), + new KeyProperty(ImmutableSet.of())); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + VariableReferenceExpression c = p.variable("c"); + return p.aggregation(builder -> builder + .singleGroupingSet(c) + .source(p.values(1, c))); + }) + .matches(expectedLogicalProperties); + + //Values has more than one row. + VariableReferenceExpression a = new VariableReferenceExpression(Optional.empty(), "a", BIGINT); + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(3L), + new KeyProperty(ImmutableSet.of())); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.values( + ImmutableList.of(a), + ImmutableList.of(ImmutableList.of(constant(1L, BIGINT)), + ImmutableList.of(constant(2L, BIGINT)), + ImmutableList.of(constant(3L, BIGINT))))) + .matches(expectedLogicalProperties); + } + + @Test + public void testTableScanNodeLogicalProperties() + { + // "custkey" should be a key in the result of TableScan(customer) + LogicalProperties expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(customerCustKeyVariable))))); + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle))) + .matches(expectedLogicalProperties); + + // add an additional unique constraint on customer (comment, nationkey)column + Set commentcolumnSet = new HashSet<>(); + commentcolumnSet.add(customerCommentColumn); + UniqueConstraint commentConstraint = new UniqueConstraint<>(commentcolumnSet, true, true); + List> customerConstraints = new ArrayList<>(tester().getTableConstraints(customerTableHandle)); + customerConstraints.add(commentConstraint); + + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(customerCustKeyVariable)), new Key(ImmutableSet.of(customerCommentVariable))))); + List> finalCustomerConstraints = customerConstraints; + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable, customerCommentVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn, customerCommentVariable, customerCommentColumn), + TupleDomain.none(), + TupleDomain.none(), + finalCustomerConstraints)) + .matches(expectedLogicalProperties); + + //TEST: the previous test but there is no assigment for the comment column + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(customerCustKeyVariable))))); + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + finalCustomerConstraints)) + .matches(expectedLogicalProperties); + + //TEST: add a superfulous unique constraint on the (custkey, comment) combination + Set custkeyCommentColumnSet = new HashSet<>(); + custkeyCommentColumnSet.add(customerCustKeyColumn); + custkeyCommentColumnSet.add(customerCommentColumn); + UniqueConstraint custkeyCommentConstraint = new UniqueConstraint<>(custkeyCommentColumnSet, true, true); + customerConstraints = new ArrayList<>(tester().getTableConstraints(customerTableHandle)); + customerConstraints.add(custkeyCommentConstraint); + + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(customerCustKeyVariable))))); + List> finalCustomerConstraints1 = customerConstraints; + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable, customerCommentVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn, + customerCommentVariable, customerCommentColumn), + TupleDomain.none(), + TupleDomain.none(), + finalCustomerConstraints1)) + .matches(expectedLogicalProperties); + + //Define a table with key (A,B) but only give a table scan mapping for A (B). The key property of the table scan should be empty. + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of())); + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + ImmutableList.of(custkeyCommentConstraint))) + .matches(expectedLogicalProperties); + + // INVARIANT: define a table with primary key (A) and unique key (A,B) and ensure that the table scan key property only has key (A) (both A and B should have mappings) + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(customerCustKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable, customerCommentVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn, + customerCommentVariable, customerCommentColumn), + TupleDomain.none(), + TupleDomain.none(), + finalCustomerConstraints1)) + .matches(expectedLogicalProperties); + + // INVARIANT: define a table with primary key (A,B) and unique key (A) and ensure that the table scan key property only has key (A) (both A and B should have mappings) + PrimaryKeyConstraint custkeyCommentPK = new PrimaryKeyConstraint<>("primarykey", custkeyCommentColumnSet, true, true); + UniqueConstraint custkeyUniqueConstraint = new UniqueConstraint<>(ImmutableSet.of(customerCustKeyColumn), true, true); + + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(customerCustKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable, customerCommentVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn, + customerCommentVariable, customerCommentColumn), + TupleDomain.none(), + TupleDomain.none(), + ImmutableList.of(custkeyCommentPK, custkeyUniqueConstraint))) + .matches(expectedLogicalProperties); + } + + @Test + void testFilterNodeLogicalProperties() + { + ConstantExpression constExpr = new ConstantExpression(100L, BIGINT); + EquivalenceClassProperty equivalenceClasses = new EquivalenceClassProperty(functionResolution); + equivalenceClasses.update(customerCustKeyVariable, constExpr); + + LogicalProperties expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(1L), + new KeyProperty(ImmutableSet.of())); + + // Primary key will be propagated till the FilterNode, where maxCardProperty will be set to 1 + // and KeyProperty will be cleared. Equivalence class property reflects the predicate "custkey = 100". + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(customerCustKeyVariable); + return p.filter(p.rowExpression("c_custkey = BIGINT '100'"), + p.tableScan( + customerTableHandle, + ImmutableList.of(acctBalVariable), + ImmutableMap.of(acctBalVariable, acctBalColumn, + customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle))); + }) + .matches(expectedLogicalProperties); + + //INVARIANT: define a table with keys (A) and (B,C) and apply predicate A=B and ensure that the filter key property only has key (A) + ColumnHandle colA = new TpchColumnHandle("A", BIGINT); + ColumnHandle colB = new TpchColumnHandle("B", BIGINT); + ColumnHandle colC = new TpchColumnHandle("C", BIGINT); + + PrimaryKeyConstraint primaryKeyConstraint = new PrimaryKeyConstraint<>("primarykey", ImmutableSet.of(colA), true, true); + UniqueConstraint uniqueConstraint = new UniqueConstraint<>(ImmutableSet.of(colB, colC), true, true); + List> tableConstraints = ImmutableList.of(primaryKeyConstraint, uniqueConstraint); + + VariableReferenceExpression varA = new VariableReferenceExpression(Optional.empty(), "A", BIGINT); + VariableReferenceExpression varB = new VariableReferenceExpression(Optional.empty(), "B", BIGINT); + VariableReferenceExpression varC = new VariableReferenceExpression(Optional.empty(), "C", BIGINT); + + equivalenceClasses = new EquivalenceClassProperty(functionResolution); + equivalenceClasses.update(varA, varB); + + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(varA))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(varA); + p.variable(varB); + return p.filter(p.rowExpression("A = B"), + p.tableScan( + customerTableHandle, + ImmutableList.of(varA), + ImmutableMap.of(varA, colA, varB, colB, varC, colC), + TupleDomain.none(), + TupleDomain.none(), + tableConstraints)); + }) + .matches(expectedLogicalProperties); + + //INVARIANT: define a table with keys (A,C) and (B,C) and apply predicate A=constant and ensure that the filter key property has only has one key (C) + PrimaryKeyConstraint primaryKeyConstraint1 = new PrimaryKeyConstraint<>("primarykey", ImmutableSet.of(colA, colC), true, true); + UniqueConstraint uniqueConstraint1 = new UniqueConstraint<>(ImmutableSet.of(colB, colC), true, true); + List> tableConstraints1 = ImmutableList.of(primaryKeyConstraint1, uniqueConstraint1); + + equivalenceClasses = new EquivalenceClassProperty(functionResolution); + equivalenceClasses.update(varA, constant(100L, BIGINT)); + + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(varC))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(varA); + p.variable(varB); + return p.filter(p.rowExpression("A = BIGINT '100'"), + p.tableScan( + customerTableHandle, + ImmutableList.of(varA), + ImmutableMap.of(varA, colA, varB, colB, varC, colC), + TupleDomain.none(), + TupleDomain.none(), + tableConstraints1)); + }) + .matches(expectedLogicalProperties); + + //INVARIANT: define a table with key (A,B) and apply predicates A=constant1 and B=constant2 ensure that the filter has maxcard=1 and key property is empty + + List> tableConstraints2 = ImmutableList.of(new PrimaryKeyConstraint<>("primarykey", ImmutableSet.of(colA, colB), true, true)); + + equivalenceClasses = new EquivalenceClassProperty(functionResolution); + equivalenceClasses.update(varA, constant(100L, BIGINT)); + equivalenceClasses.update(varB, constant(50L, BIGINT)); + + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(1L), + new KeyProperty(ImmutableSet.of())); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(varA); + p.variable(varB); + return p.filter(p.rowExpression("A = BIGINT '100' AND B = BIGINT '50'"), + p.tableScan( + customerTableHandle, + ImmutableList.of(varA), + ImmutableMap.of(varA, colA, varB, colB, varC, colC), + TupleDomain.none(), + TupleDomain.none(), + tableConstraints2)); + }) + .matches(expectedLogicalProperties); + + //INVARIANT: define a table with key (A,B) and apply predicates A=constant and A=B ensure that the filter has maxcard=1 and key property is empty + equivalenceClasses = new EquivalenceClassProperty(functionResolution); + equivalenceClasses.update(varA, varB); + equivalenceClasses.update(varA, constant(100L, BIGINT)); + + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(1L), + new KeyProperty(ImmutableSet.of())); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(varA); + p.variable(varB); + return p.filter(p.rowExpression("A = B AND A = BIGINT '100'"), + p.tableScan( + customerTableHandle, + ImmutableList.of(varA), + ImmutableMap.of(varA, colA, varB, colB, varC, colC), + TupleDomain.none(), + TupleDomain.none(), + tableConstraints2)); + }) + .matches(expectedLogicalProperties); + } + + @Test + public void testProjectNodeLogicalProperties() + { + VariableReferenceExpression projectedCustKeyVariable = new VariableReferenceExpression(Optional.empty(), "newcustkey", BIGINT); + Assignments assignments = Assignments.builder().put(projectedCustKeyVariable, customerCustKeyVariable).build(); + + LogicalProperties expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(projectedCustKeyVariable))))); + + // Test Logical properties generated for the project node + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.project(assignments, p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)))) + .matches(expectedLogicalProperties); + + //TableScan has key property (A,B). Project only has mapping A->A' and hence result key property should be empty. + ColumnHandle colA = new TpchColumnHandle("A", BIGINT); + ColumnHandle colB = new TpchColumnHandle("B", BIGINT); + VariableReferenceExpression varA = new VariableReferenceExpression(Optional.empty(), "A", BIGINT); + VariableReferenceExpression varB = new VariableReferenceExpression(Optional.empty(), "B", BIGINT); + VariableReferenceExpression projectedVarA = new VariableReferenceExpression(Optional.empty(), "A1", BIGINT); + List> tableConstraints = ImmutableList.of(new PrimaryKeyConstraint<>("primarykey", ImmutableSet.of(colA, colB), true, true)); + Assignments assignments1 = Assignments.builder().put(projectedVarA, varA).build(); + + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of())); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(varA); + p.variable(varB); + return p.project(assignments1, p.tableScan( + customerTableHandle, + ImmutableList.of(varA), + ImmutableMap.of(varA, colA, varB, colB), + TupleDomain.none(), + TupleDomain.none(), + tableConstraints)); + }) + .matches(expectedLogicalProperties); + + //TableScan key property has key (A), Filter applies predicate A=B, Project only has a mapping B->B'. Project should have key property with (B'). + List> tableConstraints1 = ImmutableList.of(new PrimaryKeyConstraint<>("primarykey", ImmutableSet.of(colA), true, true)); + VariableReferenceExpression projectedA = new VariableReferenceExpression(Optional.empty(), "A1", BIGINT); + Assignments assignments2 = Assignments.builder().put(projectedA, varA).build(); + + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(projectedA))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(varA); + p.variable(varB); + return p.project(assignments2, + p.filter(p.rowExpression("A = B"), + p.tableScan( + customerTableHandle, + ImmutableList.of(varA), + ImmutableMap.of(varA, colA, varB, colB), + TupleDomain.none(), + TupleDomain.none(), + tableConstraints1))); + }) + .matches(expectedLogicalProperties); + + // INVARIANT Filter creates multiple equivalence classes e.g. (A, B, C) (D, E, F). Test various cases where + // all or only some of these have mappings. These should include cases where the equivalence class heads are + // projected out, members are projected out, and also cases where the entire equivalence class is projected out. + ColumnHandle colC = new TpchColumnHandle("C", BIGINT); + ColumnHandle colD = new TpchColumnHandle("D", BIGINT); + ColumnHandle colE = new TpchColumnHandle("E", BIGINT); + ColumnHandle colF = new TpchColumnHandle("F", BIGINT); + VariableReferenceExpression varC = new VariableReferenceExpression(Optional.empty(), "C", BIGINT); + VariableReferenceExpression varD = new VariableReferenceExpression(Optional.empty(), "D", BIGINT); + VariableReferenceExpression varE = new VariableReferenceExpression(Optional.empty(), "E", BIGINT); + VariableReferenceExpression varF = new VariableReferenceExpression(Optional.empty(), "F", BIGINT); + VariableReferenceExpression projectedB = new VariableReferenceExpression(Optional.empty(), "B1", BIGINT); + VariableReferenceExpression projectedC = new VariableReferenceExpression(Optional.empty(), "C1", BIGINT); + VariableReferenceExpression projectedD = new VariableReferenceExpression(Optional.empty(), "D1", BIGINT); + VariableReferenceExpression projectedE = new VariableReferenceExpression(Optional.empty(), "E1", BIGINT); + VariableReferenceExpression projectedF = new VariableReferenceExpression(Optional.empty(), "F1", BIGINT); + + Map scanAssignments = + new ImmutableMap.Builder() + .put(varA, colA) + .put(varB, colB) + .put(varC, colC) + .put(varD, colD) + .put(varE, colE) + .put(varF, colF).build(); + + Assignments projectAssignments = Assignments.builder() + .put(projectedA, varA) + .put(projectedB, varB) + .put(projectedC, varC) + .put(projectedD, varD) + .put(projectedE, varE) + .put(projectedF, varF) + .build(); + + // A = B and B = C and D = E and E = F + EquivalenceClassProperty equivalenceClassProperty = new EquivalenceClassProperty(functionResolution); + equivalenceClassProperty.update(projectedA, projectedB); + equivalenceClassProperty.update(projectedB, projectedC); + equivalenceClassProperty.update(projectedD, projectedE); + equivalenceClassProperty.update(projectedE, projectedF); + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClassProperty, + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(projectedA))))); + + // ProjectNode projects all variables used in the filter. + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(varA); + p.variable(varB); + p.variable(varC); + p.variable(varD); + p.variable(varE); + p.variable(varF); + + return p.project(projectAssignments, + p.filter(p.rowExpression("A = B AND B = C AND D = E AND E = F"), + p.tableScan( + customerTableHandle, + ImmutableList.of(varA, varB, varC, varD, varE, varF), + scanAssignments, + TupleDomain.none(), + TupleDomain.none(), + tableConstraints1))); + }) + .matches(expectedLogicalProperties); + + // ProjectNode projects only equivalence class heads(A, D). Equivalence classes should be empty. KeyProperty + // should be set to A1. + Assignments projectAssignments1 = Assignments.builder() + .put(projectedA, varA) + .put(projectedD, varD) + .build(); + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(projectedA))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(varA); + p.variable(varB); + p.variable(varC); + p.variable(varD); + p.variable(varE); + p.variable(varF); + + return p.project(projectAssignments1, + p.filter(p.rowExpression("A = B AND B = C AND D = E AND E = F"), + p.tableScan( + customerTableHandle, + ImmutableList.of(varA, varB, varC, varD, varE, varF), + scanAssignments, + TupleDomain.none(), + TupleDomain.none(), + tableConstraints1))); + }) + .matches(expectedLogicalProperties); + + // ProjectNode projects only equivalence class members(B,C,E,F). Equivalence classes should have (B,C), (E,F). + // KeyProperty should have B1 + Assignments projectAssignments2 = Assignments.builder() + .put(projectedB, varB) + .put(projectedC, varC) + .put(projectedE, varE) + .put(projectedF, varF) + .build(); + EquivalenceClassProperty equivalenceClassProperty1 = new EquivalenceClassProperty(functionResolution); + equivalenceClassProperty1.update(projectedB, projectedC); + equivalenceClassProperty1.update(projectedE, projectedF); + + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClassProperty1, + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(projectedB))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(varA); + p.variable(varB); + p.variable(varC); + p.variable(varD); + p.variable(varE); + p.variable(varF); + + return p.project(projectAssignments2, + p.filter(p.rowExpression("A = B AND B = C AND D = E AND E = F"), + p.tableScan( + customerTableHandle, + ImmutableList.of(varA, varB, varC, varD, varE, varF), + scanAssignments, + TupleDomain.none(), + TupleDomain.none(), + tableConstraints1))); + }) + .matches(expectedLogicalProperties); + + // ProjectNode projects only equivalence class members(E,F). Equivalence classes should have (E,F). + // KeyProperty should become empty since A,B,C are removed from projection. + Assignments projectAssignments3 = Assignments.builder() + .put(projectedE, varE) + .put(projectedF, varF) + .build(); + EquivalenceClassProperty equivalenceClassProperty2 = new EquivalenceClassProperty(functionResolution); + equivalenceClassProperty2.update(projectedE, projectedF); + + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClassProperty2, + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of())); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(varA); + p.variable(varB); + p.variable(varC); + p.variable(varD); + p.variable(varE); + p.variable(varF); + + return p.project(projectAssignments3, + p.filter(p.rowExpression("A = B AND B = C AND D = E AND E = F"), + p.tableScan( + customerTableHandle, + ImmutableList.of(varA, varB, varC, varD, varE, varF), + scanAssignments, + TupleDomain.none(), + TupleDomain.none(), + tableConstraints1))); + }) + .matches(expectedLogicalProperties); + } + + @Test + public void testJoinNodeLogicalProperties() + { + // TEST: n to 1 inner join between orders and customers with limit 5 on left table. + // orders key property, maxcard=5 and equivalence classes(o_custkey=c_custkey), (shippriority=10) and + // (mktsegment='BUILDING') should be propagated. + EquivalenceClassProperty equivalenceClasses = new EquivalenceClassProperty(functionResolution); + equivalenceClasses.update(ordersCustKeyVariable, customerCustKeyVariable); + equivalenceClasses.update(shipPriorityVariable, constant(10L, INTEGER)); + equivalenceClasses.update(mktSegmentVariable, constant(Slices.utf8Slice("BUILDING"), createVarcharType(8))); + + LogicalProperties expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(5L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable, mktSegmentVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn, mktSegmentVariable, mktSegmentColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable, shipPriorityVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, + ordersOrderKeyVariable, ordersOrderKeyColumn, + shipPriorityVariable, shipPriorityColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + p.variable(shipPriorityVariable); + p.variable(mktSegmentVariable); + return p.join(JoinNode.Type.INNER, + p.limit(5, ordersTableScan), + p.filter(p.rowExpression("c_mktsegment = 'BUILDING'"), customerTableScan), + p.rowExpression("o_shippriority = 10"), + new JoinNode.EquiJoinClause(ordersCustKeyVariable, customerCustKeyVariable)); + }) + .matches(expectedLogicalProperties); + + // TEST: n to 1 inner join between orders and customers with limit 1 on left table. maxcard=1 and equivalence + // classes(o_custkey=c_custkey),(shippriority=10) and (mktsegment='BUILDING') should be propagated. + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(1L), + new KeyProperty(ImmutableSet.of())); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable, mktSegmentVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn, mktSegmentVariable, mktSegmentColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable, shipPriorityVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, + ordersOrderKeyVariable, ordersOrderKeyColumn, + shipPriorityVariable, shipPriorityColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + p.variable(shipPriorityVariable); + p.variable(mktSegmentVariable); + return p.join(JoinNode.Type.INNER, + p.limit(1, ordersTableScan), + p.filter(p.rowExpression("c_mktsegment = 'BUILDING'"), customerTableScan), + p.rowExpression("o_shippriority = 10"), + new JoinNode.EquiJoinClause(ordersCustKeyVariable, customerCustKeyVariable)); + }) + .matches(expectedLogicalProperties); + + // TEST: 1 to n inner join between customers and orders with limit(6) on the right table. + // orders key property and maxcard=6 and equivalence classes(o_custkey=c_custkey),(shippriority=10) and + // (mktsegment='BUILDING') should be propagated. + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(6L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable, mktSegmentVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn, mktSegmentVariable, mktSegmentColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable, shipPriorityVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, + ordersOrderKeyVariable, ordersOrderKeyColumn, + shipPriorityVariable, shipPriorityColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + p.variable(shipPriorityVariable); + p.variable(mktSegmentVariable); + return p.join(JoinNode.Type.INNER, + p.filter(p.rowExpression("c_mktsegment = 'BUILDING'"), customerTableScan), + p.limit(6, ordersTableScan), + p.rowExpression("o_shippriority = 10"), + new JoinNode.EquiJoinClause(customerCustKeyVariable, ordersCustKeyVariable)); + }) + .matches(expectedLogicalProperties); + + // TEST: 1 to n inner join between customers and orders with limit(1) on the right table. + // Only maxcard=1 should get propagated + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(1L), + new KeyProperty(ImmutableSet.of())); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable, mktSegmentVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn, mktSegmentVariable, mktSegmentColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable, shipPriorityVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, + ordersOrderKeyVariable, ordersOrderKeyColumn, + shipPriorityVariable, shipPriorityColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + p.variable(shipPriorityVariable); + p.variable(mktSegmentVariable); + return p.join(JoinNode.Type.INNER, + p.filter(p.rowExpression("c_mktsegment = 'BUILDING'"), customerTableScan), + p.limit(1, ordersTableScan), + p.rowExpression("o_shippriority = 10"), + new JoinNode.EquiJoinClause(customerCustKeyVariable, ordersCustKeyVariable)); + }) + .matches(expectedLogicalProperties); + + // TEST: n to 1 left join between orders and customers with limit(7) on the left table. + // orders keys and maxcard=7 are propagated + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(7L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + return p.join(JoinNode.Type.LEFT, p.limit(7, ordersTableScan), customerTableScan, + new JoinNode.EquiJoinClause(ordersCustKeyVariable, customerCustKeyVariable)); + }) + .matches(expectedLogicalProperties); + + // TEST: n to 1 left join between orders and customers with limit on right table. + // orders keys are propagated. Maxcard should not be propagated. + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + return p.join(JoinNode.Type.LEFT, ordersTableScan, p.limit(8, customerTableScan), + new JoinNode.EquiJoinClause(ordersCustKeyVariable, customerCustKeyVariable)); + }) + .matches(expectedLogicalProperties); + + // TEST: 1 to n right join between customers and orders. Limit(9) on the right table. + // orders keys are propagated and maxcard=9 should be propagated. + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(9L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + return p.join(JoinNode.Type.RIGHT, customerTableScan, p.limit(9, ordersTableScan), + new JoinNode.EquiJoinClause(customerCustKeyVariable, ordersCustKeyVariable)); + }) + .matches(expectedLogicalProperties); + + // TEST: 1 to n right join between customers and orders. Limit(10) on the left table. + // orders keys are propagated. maxcard should not be propagated. + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + return p.join(JoinNode.Type.RIGHT, p.limit(10, customerTableScan), ordersTableScan, + new JoinNode.EquiJoinClause(customerCustKeyVariable, ordersCustKeyVariable)); + }) + .matches(expectedLogicalProperties); + + // TEST: 1 to n left join between customers and orders - no keys are propagated + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + return p.join(JoinNode.Type.LEFT, customerTableScan, ordersTableScan, + new JoinNode.EquiJoinClause(customerCustKeyVariable, ordersCustKeyVariable)); + }) + .matches(expectedLogicalProperties); + + // TEST: n to m inner join between customers and orders - concatenated key (orderkey, custkey) should get propagated. + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable, customerCustKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + return p.join(JoinNode.Type.INNER, customerTableScan, ordersTableScan, emptyList(), ImmutableList.of(ordersOrderKeyVariable, customerCustKeyVariable, ordersCustKeyVariable), Optional.empty()); + }) + .matches(expectedLogicalProperties); + + // TEST: n to m inner join between customers and orders and limit 11 left table and limit 12 on right table. + // concatenated key (orderkey, custkey) should get propagated. Maxcard should be maxCardLeft * maxCardRight = 132. + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(132L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable, customerCustKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + return p.join(JoinNode.Type.INNER, p.limit(11, customerTableScan), p.limit(12, ordersTableScan), emptyList(), ImmutableList.of(ordersOrderKeyVariable, ordersCustKeyVariable, customerCustKeyVariable), Optional.empty()); + }) + .matches(expectedLogicalProperties); + + //test m to n cases where there are multiple keys in the left and right tables to concatenate e.g. add unique keys customer.comment and orders.comment + List> customerTableConstraints = new ArrayList<>(tester().getTableConstraints(customerTableHandle)); + customerTableConstraints.add(new UniqueConstraint<>(ImmutableSet.of(customerCommentColumn), true, true)); + + List> orderTableConstraints = new ArrayList<>(tester().getTableConstraints(ordersTableHandle)); + orderTableConstraints.add(new UniqueConstraint<>(ImmutableSet.of(ordersCommentColumn), true, true)); + + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable, customerCustKeyVariable)), + new Key(ImmutableSet.of(customerCommentVariable, ordersCommentVariable)), + new Key(ImmutableSet.of(ordersOrderKeyVariable, customerCommentVariable)), + new Key(ImmutableSet.of(customerCustKeyVariable, ordersCommentVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable, customerCommentVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn, customerCommentVariable, customerCommentColumn), + TupleDomain.none(), + TupleDomain.none(), + customerTableConstraints); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable, ordersCommentVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, + ordersOrderKeyVariable, ordersOrderKeyColumn, + ordersCommentVariable, ordersCommentColumn), + TupleDomain.none(), + TupleDomain.none(), + orderTableConstraints); + + return p.join(JoinNode.Type.INNER, customerTableScan, ordersTableScan, emptyList(), + ImmutableList.of(ordersOrderKeyVariable, ordersCustKeyVariable, ordersCommentVariable, + customerCustKeyVariable, customerCommentVariable), + Optional.empty()); + }) + .matches(expectedLogicalProperties); + + // TEST: n to m inner join between values(1) and values(1) - maxcard(1) should get propagated. + VariableReferenceExpression c1 = new VariableReferenceExpression(Optional.empty(), "c1", BIGINT); + VariableReferenceExpression c2 = new VariableReferenceExpression(Optional.empty(), "c2", BIGINT); + + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(1L), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + ValuesNode values1 = p.values(1, c1); + ValuesNode values2 = p.values(1, c2); + return p.join(JoinNode.Type.INNER, values1, values2, emptyList(), ImmutableList.of(c1, c2), Optional.empty()); + }) + .matches(expectedLogicalProperties); + + // TEST: n to m full join between values(1) and values(1) - maxcard(1) should get propagated. + VariableReferenceExpression c3 = new VariableReferenceExpression(Optional.empty(), "c1", BIGINT); + VariableReferenceExpression c4 = new VariableReferenceExpression(Optional.empty(), "c2", BIGINT); + + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(1L), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + ValuesNode values1 = p.values(1, c3); + ValuesNode values2 = p.values(1, c4); + return p.join(JoinNode.Type.FULL, values1, values2, emptyList(), ImmutableList.of(c3, c4), Optional.empty()); + }) + .matches(expectedLogicalProperties); + + // TEST: 1 to n full join between customers and orders - nothing should get propagated. + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + return p.join(JoinNode.Type.FULL, customerTableScan, ordersTableScan, emptyList(), ImmutableList.of(customerCustKeyVariable, ordersOrderKeyVariable), Optional.empty()); + }) + .matches(expectedLogicalProperties); + + // TEST: n to 1 full join between customers and orders with limit(12) on left and and limit(10) on right table. + // The product of the maxcards 120 should get propagated + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(120L), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + return p.join(JoinNode.Type.FULL, p.limit(12, customerTableScan), p.limit(10, ordersTableScan), + ImmutableList.of(new JoinNode.EquiJoinClause(customerCustKeyVariable, ordersCustKeyVariable)), + ImmutableList.of(ordersOrderKeyVariable, customerCustKeyVariable), + Optional.empty()); + }) + .matches(expectedLogicalProperties); + + // TEST: n to m full join between customers and orders with maxcard 2 on left and unknown maxcard on right table. + // Concatenated keys and a maxcard of unknown should get propagated + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable, customerCustKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + return p.join(JoinNode.Type.INNER, p.limit(2, customerTableScan), ordersTableScan, emptyList(), + ImmutableList.of(ordersOrderKeyVariable, ordersCustKeyVariable, + customerCustKeyVariable), + Optional.empty()); + }) + .matches(expectedLogicalProperties); + + // TEST: n to m full join between customers and orders with maxcard 2 on left and unknown maxcard on right table. + // Concatenated keys and a maxcard of unknown should get propagated + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable, customerCustKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + return p.join(JoinNode.Type.INNER, ordersTableScan, p.limit(2, customerTableScan), emptyList(), + ImmutableList.of(ordersOrderKeyVariable, ordersCustKeyVariable, + customerCustKeyVariable), + Optional.empty()); + }) + .matches(expectedLogicalProperties); + + // TEST: 1 to 1 inner join between values(1) and customers - maxcard(1) is propagated + equivalenceClasses = new EquivalenceClassProperty(functionResolution); + VariableReferenceExpression c = new VariableReferenceExpression(Optional.empty(), "c", BIGINT); + equivalenceClasses.update(c, customerCustKeyVariable); + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(1L), + new KeyProperty()); + + VariableReferenceExpression finalC1 = c; + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + ValuesNode values = p.values(1, finalC1); + + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + return p.join(JoinNode.Type.INNER, values, customerTableScan, + new JoinNode.EquiJoinClause(finalC1, customerCustKeyVariable)); + }) + .matches(expectedLogicalProperties); + + // TEST: 1 to 1 inner join between customers and values(1) - maxcard(1) is propagated + equivalenceClasses = new EquivalenceClassProperty(functionResolution); + c = new VariableReferenceExpression(Optional.empty(), "c", BIGINT); + equivalenceClasses.update(c, customerCustKeyVariable); + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(1L), + new KeyProperty()); + + VariableReferenceExpression finalC = c; + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + ValuesNode values = p.values(1, finalC); + + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + return p.join(JoinNode.Type.INNER, customerTableScan, values, + new JoinNode.EquiJoinClause(customerCustKeyVariable, finalC)); + }) + .matches(expectedLogicalProperties); + + // TEST: 1 to n full join between customers and values(1) where n=1 - maxcard(1) is propagated + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(1L), + new KeyProperty()); + + VariableReferenceExpression finalC2 = c; + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + ValuesNode values = p.values(1, finalC2); + + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + return p.join(JoinNode.Type.FULL, customerTableScan, values, + new JoinNode.EquiJoinClause(customerCustKeyVariable, finalC2)); + }) + .matches(expectedLogicalProperties); + + // TEST: n to 1 full join between values(1) and customers where n=1 - maxcard(1) is propagated + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(1L), + new KeyProperty()); + + VariableReferenceExpression finalC3 = c; + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + ValuesNode values = p.values(1, finalC3); + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + return p.join(JoinNode.Type.FULL, values, customerTableScan, + new JoinNode.EquiJoinClause(finalC3, customerCustKeyVariable)); + }) + .matches(expectedLogicalProperties); + + // Three table join. Key (l_orderkey, l_linenumber), maxCard=6 and equivalence classes (o_orderkey,l_orderkey) and + // (o_custkey, c_custkey) should be propagated. + equivalenceClasses = new EquivalenceClassProperty(functionResolution); + equivalenceClasses.update(customerCustKeyVariable, ordersCustKeyVariable); + equivalenceClasses.update(ordersOrderKeyVariable, lineitemOrderkeyVariable); + + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(lineitemLinenumberVariable, ordersOrderKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable, mktSegmentVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn, mktSegmentVariable, mktSegmentColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable, shipPriorityVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, + ordersOrderKeyVariable, ordersOrderKeyColumn, + shipPriorityVariable, shipPriorityColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + TableScanNode lineitemTableScan = p.tableScan( + lineitemTableHandle, + ImmutableList.of(lineitemLinenumberVariable, lineitemOrderkeyVariable), + ImmutableMap.of(lineitemLinenumberVariable, lineitemLinenumberColumn, + lineitemOrderkeyVariable, lineitemOrderkeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(lineitemTableHandle)); + + JoinNode customerOrderJoin = p.join(JoinNode.Type.INNER, + customerTableScan, + p.limit(6, ordersTableScan), + new JoinNode.EquiJoinClause(customerCustKeyVariable, ordersCustKeyVariable)); + + return p.join(JoinNode.Type.INNER, + customerOrderJoin, + lineitemTableScan, + new JoinNode.EquiJoinClause(ordersOrderKeyVariable, lineitemOrderkeyVariable)); + }) + .matches(expectedLogicalProperties); + } + + @Test + public void testSemiJoinNodeLogicalProperties() + { + EquivalenceClassProperty equivalenceClasses = new EquivalenceClassProperty(functionResolution); + equivalenceClasses.update(ordersCustKeyVariable, customerCustKeyVariable); + + LogicalProperties expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable))))); + + // Test Logical properties generated for semi join node. It just propagates its non-filtering source's properties. + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + return p.semiJoin(ordersCustKeyVariable, customerCustKeyVariable, ordersOrderKeyVariable, + Optional.empty(), Optional.empty(), ordersTableScan, customerTableScan); + }) + .matches(expectedLogicalProperties); + + //source table is 1-tuple + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(1L), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + ValuesNode semiJoinSource = p.values(1, ordersCustKeyVariable); + TableScanNode semiJoinFilteringSource = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + return p.semiJoin(ordersCustKeyVariable, customerCustKeyVariable, ordersOrderKeyVariable, + Optional.empty(), Optional.empty(), semiJoinSource, semiJoinFilteringSource); + }) + .matches(expectedLogicalProperties); + + //maxcard derived from limit propagates semijoin + EquivalenceClassProperty equivalenceClassProperty = new EquivalenceClassProperty(functionResolution); + equivalenceClassProperty.update(ordersOrderPriorityVariable, constant(Slices.utf8Slice("URGENT"), createVarcharType(6))); + + expectedLogicalProperties = new LogicalPropertiesImpl(equivalenceClassProperty, + new MaxCardProperty(5L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode semiJoinFilteringSource = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + p.variable(ordersOrderPriorityVariable); + LimitNode semiJoinSource = p.limit(5, + p.filter(p.rowExpression("o_orderpriority = 'URGENT'"), + p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, + ordersOrderKeyVariable, ordersOrderKeyColumn, + ordersOrderPriorityVariable, ordersOrderPriorityColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)))); + + return p.semiJoin(ordersCustKeyVariable, customerCustKeyVariable, ordersOrderKeyVariable, + Optional.empty(), Optional.empty(), semiJoinSource, semiJoinFilteringSource); + }) + .matches(expectedLogicalProperties); + } + + @Test + public void testAggregationNodeLogicalProperties() + { + // Aggregation node adds new key (nationkey) + LogicalProperties expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(customerNationKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.aggregation(builder -> builder + .singleGroupingSet(customerNationKeyVariable) + .source(p.tableScan( + customerTableHandle, + ImmutableList.of(customerNationKeyVariable), + ImmutableMap.of(customerNationKeyVariable, customerNationKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + emptyList())))) + .matches(expectedLogicalProperties); + + //INVARIANT: Grouping on (nationkey, custkey) but (custkey) is already a key. So grouping result should have only (custkey) + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(customerCustKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.aggregation(builder -> builder + .singleGroupingSet(customerCustKeyVariable, customerNationKeyVariable) + .source(p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle))))) + .matches(expectedLogicalProperties); + + //INVARIANT. Group by nationkey Filter binds nationkey to a constant before grouping. Result should have maxcard=1; + EquivalenceClassProperty equivalenceClassProperty = new EquivalenceClassProperty(functionResolution); + equivalenceClassProperty.update(customerNationKeyVariable, constant(20L, BIGINT)); + expectedLogicalProperties = new LogicalPropertiesImpl(equivalenceClassProperty, + new MaxCardProperty(1L), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(customerNationKeyVariable); + return p.aggregation(builder -> builder + .singleGroupingSet(customerNationKeyVariable) + .source(p.filter(p.rowExpression("nationkey = BIGINT '20'"), + p.tableScan( + customerTableHandle, + ImmutableList.of(customerNationKeyVariable), + ImmutableMap.of(customerNationKeyVariable, customerNationKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + emptyList())))); + }) + .matches(expectedLogicalProperties); + + //INVARIANT. Group on (nationkey, mktsegment) and after first binding "mktsegment" to a constant. The grouping result should have key (nationkey) + EquivalenceClassProperty equivalenceClassProperty1 = new EquivalenceClassProperty(functionResolution); + equivalenceClassProperty1.update(mktSegmentVariable, constant(Slices.utf8Slice("BUILDING"), createVarcharType(8))); + expectedLogicalProperties = new LogicalPropertiesImpl(equivalenceClassProperty1, + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(customerNationKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(mktSegmentVariable); + return p.aggregation(builder -> builder + .singleGroupingSet(customerNationKeyVariable, mktSegmentVariable) + .source(p.filter(p.rowExpression("c_mktsegment = 'BUILDING'"), + p.tableScan( + customerTableHandle, + ImmutableList.of(customerNationKeyVariable), + ImmutableMap.of(customerNationKeyVariable, customerNationKeyColumn, mktSegmentVariable, mktSegmentColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle))))); + }) + .matches(expectedLogicalProperties); + + //INVARIANT. Group by with aggregate functions but no grouping columns. Maxard should be 1 and no keys propagated. + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(1L), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(customerNationKeyVariable); + return p.aggregation(builder -> builder + .addAggregation(p.variable("count_nk"), p.rowExpression("count(nationkey)")) + .globalGrouping() + .source(p.tableScan( + customerTableHandle, + ImmutableList.of(customerNationKeyVariable), + ImmutableMap.of(customerNationKeyVariable, customerNationKeyColumn, mktSegmentVariable, mktSegmentColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)))); + }) + .matches(expectedLogicalProperties); + + //INVARIANT. Maxcard is set to 1 prior to group by with grouping columns. Maxard of group by should be 1 and no keys propagated. + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(1L), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(customerNationKeyVariable); + return p.aggregation(builder -> builder + .singleGroupingSet(customerNationKeyVariable) + .source(p.limit(1, p.tableScan( + customerTableHandle, + ImmutableList.of(customerNationKeyVariable), + ImmutableMap.of(customerNationKeyVariable, customerNationKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + emptyList())))); + }) + .matches(expectedLogicalProperties); + + // Test propagation of equivalence classes through aggregation. + // None of the equivalence classes from aggregation's source node should be propagated since none of the + // members are projected by the aggregation node. + // Key property (shippriority, linenumber) which form the group by keys and maxcard=6 should be propagated. + + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(shipPriorityVariable, lineitemLinenumberVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable, mktSegmentVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn, mktSegmentVariable, mktSegmentColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable, shipPriorityVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, + ordersOrderKeyVariable, ordersOrderKeyColumn, + shipPriorityVariable, shipPriorityColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + TableScanNode lineitemTableScan = p.tableScan( + lineitemTableHandle, + ImmutableList.of(lineitemLinenumberVariable, lineitemOrderkeyVariable), + ImmutableMap.of(lineitemLinenumberVariable, lineitemLinenumberColumn, + lineitemOrderkeyVariable, lineitemOrderkeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(lineitemTableHandle)); + + JoinNode customerOrderJoin = p.join(JoinNode.Type.INNER, + customerTableScan, + p.limit(6, ordersTableScan), + new JoinNode.EquiJoinClause(customerCustKeyVariable, ordersCustKeyVariable)); + + p.variable(lineitemExtendedPriceVariable); + return p.aggregation(builder -> builder + .addAggregation(p.variable("sum_price", DOUBLE), p.rowExpression("sum(l_extendedprice)")) + .singleGroupingSet(lineitemLinenumberVariable, shipPriorityVariable) + .source(p.join(JoinNode.Type.INNER, + customerOrderJoin, + lineitemTableScan, + new JoinNode.EquiJoinClause(ordersOrderKeyVariable, lineitemOrderkeyVariable)))); + }) + .matches(expectedLogicalProperties); + + // A variation to the above case, where in groupby keys are (l_lineitem,o_orderkey,shippriority). Since + // (o_orderkey, l_lineitem) are already a key, the key should be normalized to have only (o_orderkey, l_lineitem). + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable, lineitemLinenumberVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable, mktSegmentVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn, mktSegmentVariable, mktSegmentColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable, shipPriorityVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, + ordersOrderKeyVariable, ordersOrderKeyColumn, + shipPriorityVariable, shipPriorityColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + TableScanNode lineitemTableScan = p.tableScan( + lineitemTableHandle, + ImmutableList.of(lineitemLinenumberVariable, lineitemOrderkeyVariable), + ImmutableMap.of(lineitemLinenumberVariable, lineitemLinenumberColumn, + lineitemOrderkeyVariable, lineitemOrderkeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(lineitemTableHandle)); + + JoinNode customerOrderJoin = p.join(JoinNode.Type.INNER, + customerTableScan, + p.limit(6, ordersTableScan), + new JoinNode.EquiJoinClause(customerCustKeyVariable, ordersCustKeyVariable)); + + p.variable(lineitemExtendedPriceVariable); + return p.aggregation(builder -> builder + .addAggregation(p.variable("sum_price", DOUBLE), p.rowExpression("sum(l_extendedprice)")) + .singleGroupingSet(lineitemLinenumberVariable, ordersOrderKeyVariable, shipPriorityVariable) + .source(p.join(JoinNode.Type.INNER, + customerOrderJoin, + lineitemTableScan, + new JoinNode.EquiJoinClause(ordersOrderKeyVariable, lineitemOrderkeyVariable)))); + }) + .matches(expectedLogicalProperties); + } + + @Test + void testAssignUniqueIdNodeLogicalProperties() + { + VariableReferenceExpression c = new VariableReferenceExpression(Optional.empty(), "c", BIGINT); + VariableReferenceExpression unique = new VariableReferenceExpression(Optional.empty(), "unique", BIGINT); + + LogicalProperties expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(5L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(unique))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.assignUniqueId(unique, + p.values(5, c))) + .matches(expectedLogicalProperties); + } + + @Test + void testDistinctLimitNodeLogicalProperties() + { + VariableReferenceExpression c = new VariableReferenceExpression(Optional.empty(), "c", BIGINT); + + LogicalProperties expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(3L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(c))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.distinctLimit(3, ImmutableList.of(c), p.values(5, c))) + .matches(expectedLogicalProperties); + + //Tests where where DistinctLimit adds a key! Mirror the aggregation tests. + + // DistinctLimit node adds new key (nationkey) + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(5L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(customerNationKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.distinctLimit(5, ImmutableList.of(customerNationKeyVariable), + p.tableScan( + customerTableHandle, + ImmutableList.of(customerNationKeyVariable), + ImmutableMap.of(customerNationKeyVariable, customerNationKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + emptyList()))) + .matches(expectedLogicalProperties); + + //INVARIANT: DistinctLimit on (nationkey, custkey) but (custkey) is already a key. So grouping result should have only (custkey) + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(6L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(customerCustKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.distinctLimit(6, ImmutableList.of(customerCustKeyVariable, customerNationKeyVariable), + p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)))) + .matches(expectedLogicalProperties); + + //INVARIANT. DistinctLimit with nationkey as distinct symbol. + // Filter binds nationkey to a constant before grouping. Result should have maxcard=1; + EquivalenceClassProperty equivalenceClassProperty = new EquivalenceClassProperty(functionResolution); + equivalenceClassProperty.update(customerNationKeyVariable, constant(20L, BIGINT)); + expectedLogicalProperties = new LogicalPropertiesImpl(equivalenceClassProperty, + new MaxCardProperty(1L), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(customerNationKeyVariable); + return p.distinctLimit(6, ImmutableList.of(customerNationKeyVariable), + p.filter(p.rowExpression("nationkey = BIGINT '20'"), + p.tableScan( + customerTableHandle, + ImmutableList.of(customerNationKeyVariable), + ImmutableMap.of(customerNationKeyVariable, customerNationKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + emptyList()))); + }) + .matches(expectedLogicalProperties); + + //INVARIANT. DistinctLimit with (nationkey, mktsegment) as symbols and after first binding "mktsegment" to a constant. + // The grouping result should have key (nationkey) + EquivalenceClassProperty equivalenceClassProperty1 = new EquivalenceClassProperty(functionResolution); + equivalenceClassProperty1.update(mktSegmentVariable, constant(Slices.utf8Slice("BUILDING"), createVarcharType(8))); + expectedLogicalProperties = new LogicalPropertiesImpl(equivalenceClassProperty1, + new MaxCardProperty(7L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(customerNationKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(mktSegmentVariable); + return p.distinctLimit(7, ImmutableList.of(customerNationKeyVariable, mktSegmentVariable), + p.filter(p.rowExpression("c_mktsegment = 'BUILDING'"), + p.tableScan( + customerTableHandle, + ImmutableList.of(customerNationKeyVariable), + ImmutableMap.of(customerNationKeyVariable, customerNationKeyColumn, mktSegmentVariable, mktSegmentColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)))); + }) + .matches(expectedLogicalProperties); + + //INVARIANT. Maxcard is set to 1 prior to distinct limit. Maxard of distinct limit should be 1 and no keys propagated. + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(1L), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(customerNationKeyVariable); + return p.distinctLimit(8, ImmutableList.of(customerNationKeyVariable), + p.limit(1, p.tableScan( + customerTableHandle, + ImmutableList.of(customerNationKeyVariable), + ImmutableMap.of(customerNationKeyVariable, customerNationKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + emptyList()))); + }) + .matches(expectedLogicalProperties); + + //test cases where the DistinctLimit count is 1 and results in maxcard 1 + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(1L), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(customerNationKeyVariable); + return p.distinctLimit(1, ImmutableList.of(customerNationKeyVariable), + p.tableScan( + customerTableHandle, + ImmutableList.of(customerNationKeyVariable), + ImmutableMap.of(customerNationKeyVariable, customerNationKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + emptyList())); + }) + .matches(expectedLogicalProperties); + } + + @Test + public void testLimitNodeLogicalProperties() + { + EquivalenceClassProperty equivalenceClasses = new EquivalenceClassProperty(functionResolution); + equivalenceClasses.update(ordersCustKeyVariable, customerCustKeyVariable); + + LogicalProperties expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(6L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable))))); + + // Test Logical properties generated for the Limit node. It updates the MaxCardProperty from the source properties. + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + JoinNode ordersCustomerJoin = p.join(JoinNode.Type.INNER, ordersTableScan, customerTableScan, + new JoinNode.EquiJoinClause(ordersCustKeyVariable, customerCustKeyVariable)); + + return p.limit(6, ordersCustomerJoin); + }) + .matches(expectedLogicalProperties); + + //Do a variation of the previous test where the innerjoin(TopN(orders), customer). The maxcard(5) should propagate. + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(5L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + return p.join(JoinNode.Type.INNER, p.limit(5, ordersTableScan), customerTableScan, + new JoinNode.EquiJoinClause(ordersCustKeyVariable, customerCustKeyVariable)); + }) + .matches(expectedLogicalProperties); + + //INVARIANT: maxcard is set to K (by Values or Filter) and TopN and/or Limit comes along and tries to set it to N>K. Should still be set to K. + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(5L), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.limit(10, p.values(5, p.variable("c")))) + .matches(expectedLogicalProperties); + + //INVARIANT: maxcard is set to K (by Values or Filter) and TopN and/or Limit comes along and tries to set it to N p.limit(6, p.values(10, p.variable("c")))) + .matches(expectedLogicalProperties); + + //INVARIANT: TableScan with key (A) and TopN and/or Limit sets result N=1. Key property should be emptied. + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(1L), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + return p.limit(1, customerTableScan); + }) + .matches(expectedLogicalProperties); + } + + @Test + public void testTopNNodeLogicalProperties() + { + //just duplicate the comprehensive limit tests but also do negative tests for the case where TopN is not final + EquivalenceClassProperty equivalenceClasses = new EquivalenceClassProperty(functionResolution); + equivalenceClasses.update(ordersCustKeyVariable, customerCustKeyVariable); + + LogicalProperties expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(6L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable))))); + + // Test Logical properties generated for the TopN node. It updates the MaxCardProperty from the source properties. + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + JoinNode ordersCustomerJoin = p.join(JoinNode.Type.INNER, ordersTableScan, customerTableScan, + new JoinNode.EquiJoinClause(ordersCustKeyVariable, customerCustKeyVariable)); + + return p.topN(6, ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ordersCustomerJoin); + }) + .matches(expectedLogicalProperties); + + //Variation of the previous test where the innerjoin(TopN(orders), customer). The maxcard(5) should propagate. + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(5L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + return p.join(JoinNode.Type.INNER, p.topN(5, ImmutableList.of(ordersCustKeyVariable), ordersTableScan), customerTableScan, + new JoinNode.EquiJoinClause(ordersCustKeyVariable, customerCustKeyVariable)); + }) + .matches(expectedLogicalProperties); + + //INVARIANT: maxcard is set to K (by Values or Filter) and TopN comes along and tries to set it to N>K. Should still be set to K. + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(5L), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + VariableReferenceExpression var = p.variable("c"); + return p.topN(10, ImmutableList.of(var), p.values(5, var)); + }) + .matches(expectedLogicalProperties); + + //INVARIANT: maxcard is set to K (by Values or Filter) and TopN and/or Limit comes along and tries to set it to N { + VariableReferenceExpression var = p.variable("c"); + return p.topN(6, ImmutableList.of(var), p.values(10, var)); + }) + .matches(expectedLogicalProperties); + + //INVARIANT: TableScan with key (A) and TopN and/or Limit sets result N=1. Key property should be emptied. + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(1L), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + return p.topN(1, ImmutableList.of(customerCustKeyVariable), customerTableScan); + }) + .matches(expectedLogicalProperties); + } + + @Test + void testSortNodeLogicalProperties() + { + // Test KeyProperty propagation through sort. + LogicalProperties expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(customerCustKeyVariable))))); + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode tableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + return p.sort(ImmutableList.of(customerCustKeyVariable), tableScan); + }) + .matches(expectedLogicalProperties); + + //TEST: Propagate maxcard through the filter below the sort + ConstantExpression constExpr = new ConstantExpression(100L, BIGINT); + EquivalenceClassProperty equivalenceClasses = new EquivalenceClassProperty(functionResolution); + equivalenceClasses.update(customerCustKeyVariable, constExpr); + + expectedLogicalProperties = new LogicalPropertiesImpl(equivalenceClasses, + new MaxCardProperty(1L), + new KeyProperty()); + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode tableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + p.variable(customerCustKeyVariable); + FilterNode filterNode = p.filter(p.rowExpression("c_custkey = BIGINT '100'"), tableScan); + return p.sort(ImmutableList.of(customerCustKeyVariable), filterNode); + }) + .matches(expectedLogicalProperties); + } + + @Test + public void testDefaultLogicalProperties() + { + LogicalProperties defaultLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty()); + + //Union node should not propagate any logical properties + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.union( + ImmutableListMultimap.builder() + .putAll(p.variable("o1"), p.variable("s1_c1"), p.variable("s2_c1")) + .putAll(p.variable("o2"), p.variable("s1_c2"), p.variable("s2_c2")) + .build(), + ImmutableList.of( + p.values(1, p.variable("s1_c1"), p.variable("s1_c2")), + p.values(2, p.variable("s2_c1"), p.variable("s2_c2"))))) + .matches(defaultLogicalProperties); + + //Intersect node should not propagate any logical properties + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.intersect( + ImmutableListMultimap.builder() + .putAll(p.variable("o1"), p.variable("s1_c1"), p.variable("s2_c1")) + .putAll(p.variable("o2"), p.variable("s1_c2"), p.variable("s2_c2")) + .build(), + ImmutableList.of( + p.values(1, p.variable("s1_c1"), p.variable("s1_c2")), + p.values(2, p.variable("s2_c1"), p.variable("s2_c2"))))) + .matches(defaultLogicalProperties); + + //Lateral node should not propagate any logical properties + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.lateral( + ImmutableList.of(p.variable("a")), + p.values(p.variable("a")), + p.values(p.variable("a")))) + .matches(defaultLogicalProperties); + + //MarkDistinct node should not propagate any logical properties + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + VariableReferenceExpression key = p.variable("key"); + VariableReferenceExpression mark = p.variable("mark"); + return p.markDistinct(mark, ImmutableList.of(key), p.values(10, key)); + }) + .matches(defaultLogicalProperties); + } + + private static class NoOpRule + implements Rule + { + private final Pattern pattern = Pattern.any(); + + @Override + public Pattern getPattern() + { + return pattern; + } + + @Override + public Result apply(PlanNode node, Captures captures, Context context) + { + return Result.ofPlanNode(node); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("pattern", pattern) + .toString(); + } + } +} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestRedundantAggregateDistinctRemoval.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestRedundantAggregateDistinctRemoval.java new file mode 100644 index 0000000000000..8b2f882eb5184 --- /dev/null +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestRedundantAggregateDistinctRemoval.java @@ -0,0 +1,174 @@ +/* + * 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 com.facebook.presto.sql.planner.iterative.rule; + +import com.facebook.presto.spi.plan.ProjectNode; +import com.facebook.presto.spi.relation.VariableReferenceExpression; +import com.facebook.presto.sql.planner.TestTableConstraintsConnectorFactory; +import com.facebook.presto.sql.planner.assertions.ExpectedValueProvider; +import com.facebook.presto.sql.planner.iterative.properties.LogicalPropertiesProviderImpl; +import com.facebook.presto.sql.planner.iterative.rule.test.BaseRuleTest; +import com.facebook.presto.sql.planner.iterative.rule.test.RuleTester; +import com.facebook.presto.sql.planner.plan.JoinNode; +import com.facebook.presto.sql.relational.FunctionResolution; +import com.facebook.presto.sql.tree.FunctionCall; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import java.util.Map; +import java.util.Optional; + +import static com.facebook.presto.SystemSessionProperties.USE_MARK_DISTINCT; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.aggregation; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.anySymbol; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.anyTree; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.equiJoinClause; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.functionCall; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.join; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.node; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.output; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.tableScan; +import static java.util.Collections.emptyList; + +public class TestRedundantAggregateDistinctRemoval + extends BaseRuleTest +{ + private LogicalPropertiesProviderImpl logicalPropertiesProvider; + + @BeforeClass + public final void setUp() + { + tester = new RuleTester(emptyList(), ImmutableMap.of(USE_MARK_DISTINCT, Boolean.toString(false), "exploit_constraints", Boolean.toString(true)), Optional.of(1), new TestTableConstraintsConnectorFactory(1)); + logicalPropertiesProvider = new LogicalPropertiesProviderImpl(new FunctionResolution(tester.getMetadata().getFunctionAndTypeManager())); + } + + @Test + public void singleTableTests() + { + Map> aggregations; + aggregations = ImmutableMap.of( + "count", functionCall("count", false, ImmutableList.of(anySymbol())), + "avg", functionCall("avg", true, ImmutableList.of(anySymbol()))); + + //linenumber, orderkey is a unique key, so count's distinct can be removed + tester().assertThat(ImmutableSet.of(new RemoveRedundantAggregateDistinct()), logicalPropertiesProvider) + .on("SELECT count(distinct linenumber), avg(distinct tax) FROM lineitem GROUP BY orderkey") + .matches(output( + node(ProjectNode.class, + aggregation(aggregations, + tableScan("lineitem"))))); + + aggregations = ImmutableMap.of( + "count", functionCall("count", false, ImmutableList.of(anySymbol())), + "avg", functionCall("avg", false, ImmutableList.of(anySymbol()))); + + //single row per group, so all distinct's are redundant + tester().assertThat(ImmutableSet.of(new RemoveRedundantAggregateDistinct()), logicalPropertiesProvider) + .on("SELECT count(distinct quantity), avg(distinct tax) FROM lineitem GROUP BY orderkey, linenumber") + .matches(output( + node(ProjectNode.class, + aggregation(aggregations, + tableScan("lineitem"))))); + + aggregations = ImmutableMap.of( + "count", functionCall("count", false, ImmutableList.of(anySymbol())), + "avg", functionCall("avg", false, ImmutableList.of(anySymbol()))); + + //single row input to aggregation, so all distinct's are redundant + tester().assertThat(ImmutableSet.of(new RemoveRedundantAggregateDistinct()), logicalPropertiesProvider) + .on("SELECT count(distinct quantity), avg(distinct tax) FROM lineitem where orderkey = 10 and linenumber=100 GROUP BY orderkey") + .matches(output( + node(ProjectNode.class, + aggregation(aggregations, + anyTree(tableScan("lineitem")))))); + + // Negative test cases + aggregations = ImmutableMap.of( + "count", functionCall("count", true, ImmutableList.of(anySymbol())), + "avg", functionCall("avg", true, ImmutableList.of(anySymbol()))); + + // Neither {quantity, orderkey} or {tax, orderkey} is a unique key. Note that the output of the aggregation is a single record but the input is not + tester().assertThat(ImmutableSet.of(new RemoveRedundantAggregateDistinct()), logicalPropertiesProvider) + .on("SELECT count(distinct quantity), avg(distinct tax) FROM lineitem where orderkey = 10 GROUP BY orderkey") + .matches(output( + node(ProjectNode.class, + aggregation(aggregations, + anyTree(tableScan("lineitem")))))); + + aggregations = ImmutableMap.of( + "count", functionCall("count", false, ImmutableList.of(anySymbol())), + "avg", functionCall("avg", true, ImmutableList.of(anySymbol()))); + + //RemoveRedundantAggregateDistinct is not supplied + tester().assertThat(ImmutableSet.of(), logicalPropertiesProvider) + .on("SELECT count(distinct linenumber), avg(distinct tax) FROM lineitem GROUP BY orderkey") + .doesNotMatch(output( + node(ProjectNode.class, + aggregation(aggregations, + tableScan("lineitem"))))); + } + + @Test + public void complexQueryTests() + { + Map> aggregations = ImmutableMap.of("count", functionCall("count", false, ImmutableList.of(anySymbol()))); + tester().assertThat(ImmutableSet.of(new MergeLimitWithDistinct(), new RemoveRedundantAggregateDistinct()), logicalPropertiesProvider) + .on("select count(distinct totalprice) from orders o inner join customer c on o.custkey = c.custkey group by orderkey") + .matches(output(anyTree( + aggregation( + aggregations, + join(JoinNode.Type.INNER, + ImmutableList.of(equiJoinClause("custkey", "custkey_0")), + tableScan("orders", ImmutableMap.of("totalprice", "totalprice", "orderkey", "orderkey", "custkey", "custkey")), + tableScan("customer", ImmutableMap.of("custkey_0", "custkey"))))))); + } + + @Test + public void doesNotFire() + { + tester().assertThat(new RemoveRedundantDistinct(), logicalPropertiesProvider) + .on(p -> { + VariableReferenceExpression c = p.variable("c"); + return p.aggregation(builder -> builder + .singleGroupingSet(c) + .source(p.limit( + 5, + p.values(5, c)))); + }) + .doesNotFire(); + } + + @Test + public void testFeatureDisabled() + { + // Disable the feature and verify that optimization rule is not applied. + RuleTester newTester = new RuleTester(emptyList(), ImmutableMap.of(USE_MARK_DISTINCT, Boolean.toString(false), "exploit_constraints", Boolean.toString(false))); + + Map> aggregations; + aggregations = ImmutableMap.of( + "count", functionCall("count", true, ImmutableList.of(anySymbol())), + "avg", functionCall("avg", true, ImmutableList.of(anySymbol()))); + + //Rule is not applied, so the distinct should be true for both aggregations. + newTester.assertThat(ImmutableSet.of(new RemoveRedundantAggregateDistinct()), logicalPropertiesProvider) + .on("SELECT count(distinct linenumber), avg(distinct tax) FROM lineitem GROUP BY orderkey") + .matches(output( + node(ProjectNode.class, + aggregation(aggregations, + tableScan("lineitem"))))); + } +} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestRedundantDistinctLimitRemoval.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestRedundantDistinctLimitRemoval.java new file mode 100644 index 0000000000000..595c718f60306 --- /dev/null +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestRedundantDistinctLimitRemoval.java @@ -0,0 +1,119 @@ +/* + * 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 com.facebook.presto.sql.planner.iterative.rule; + +import com.facebook.presto.spi.plan.DistinctLimitNode; +import com.facebook.presto.spi.plan.ValuesNode; +import com.facebook.presto.sql.planner.TestTableConstraintsConnectorFactory; +import com.facebook.presto.sql.planner.iterative.properties.LogicalPropertiesProviderImpl; +import com.facebook.presto.sql.planner.iterative.rule.test.BaseRuleTest; +import com.facebook.presto.sql.planner.iterative.rule.test.RuleTester; +import com.facebook.presto.sql.relational.FunctionResolution; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import java.util.Optional; + +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.anyTree; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.node; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.output; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.tableScan; +import static java.util.Collections.emptyList; + +public class TestRedundantDistinctLimitRemoval + extends BaseRuleTest +{ + private LogicalPropertiesProviderImpl logicalPropertiesProvider; + + @BeforeClass + public final void setUp() + { + tester = new RuleTester(emptyList(), ImmutableMap.of("exploit_constraints", Boolean.toString(true)), Optional.of(1), new TestTableConstraintsConnectorFactory(1)); + logicalPropertiesProvider = new LogicalPropertiesProviderImpl(new FunctionResolution(tester.getMetadata().getFunctionAndTypeManager())); + } + + @Test + public void singleTableTests() + { + tester().assertThat(new RemoveRedundantDistinctLimit(), logicalPropertiesProvider) + .on(p -> + p.distinctLimit( + 10, + ImmutableList.of(p.variable("c")), + p.values(1, p.variable("c")))) + .matches(node(ValuesNode.class)); + + tester().assertThat(new RemoveRedundantDistinctLimit(), logicalPropertiesProvider) + .on(p -> + p.distinctLimit( + 0, + ImmutableList.of(p.variable("c")), + p.values(1, p.variable("c")))) + .matches(node(ValuesNode.class)); + + tester().assertThat(ImmutableSet.of(new MergeLimitWithDistinct(), new RemoveRedundantDistinctLimit()), logicalPropertiesProvider) + .on("SELECT distinct(c) FROM (SELECT count(*) as c FROM orders) LIMIT 10") + .validates(plan -> assertNodeRemovedFromPlan(plan, DistinctLimitNode.class)); + + //negative test + tester().assertThat(ImmutableSet.of(new MergeLimitWithDistinct(), new RemoveRedundantDistinctLimit()), logicalPropertiesProvider) + .on("SELECT distinct(c) FROM (SELECT count(*) as c FROM orders GROUP BY orderkey) LIMIT 10") + .matches(output( + node(DistinctLimitNode.class, + anyTree( + tableScan("orders"))))); + + //TODO where are the constraints use cases?! + } + + @Test + public void complexQueryTests() + { + //TODO more join, complex query tests + tester().assertThat(ImmutableSet.of(new MergeLimitWithDistinct(), new RemoveRedundantDistinctLimit()), logicalPropertiesProvider) + .on("select distinct totalprice from orders o inner join customer c on o.custkey = c.custkey where o.orderkey=10 limit 2") + .validates(plan -> assertNodeRemovedFromPlan(plan, DistinctLimitNode.class)); + + //negative tests + tester().assertThat(ImmutableSet.of(new MergeLimitWithDistinct(), new RemoveRedundantDistinctLimit()), logicalPropertiesProvider) + .on("select distinct totalprice from orders o inner join customer c on o.custkey = c.custkey where o.orderkey>10 limit 2") + .validates(plan -> assertNodePresentInPlan(plan, DistinctLimitNode.class)); + } + + @Test + public void doesNotFire() + { + tester().assertThat(new RemoveRedundantDistinctLimit(), logicalPropertiesProvider) + .on(p -> + p.distinctLimit( + 10, + ImmutableList.of(p.variable("c")), + p.values(10, p.variable("c")))) + .doesNotFire(); + } + + @Test + public void testFeatureDisabled() + { + // Disable the feature and verify that optimization rule is not applied. + RuleTester newTester = new RuleTester(emptyList(), ImmutableMap.of("exploit_constraints", Boolean.toString(false))); + + newTester.assertThat(ImmutableSet.of(new MergeLimitWithDistinct(), new RemoveRedundantDistinctLimit()), logicalPropertiesProvider) + .on("select distinct totalprice from orders o inner join customer c on o.custkey = c.custkey where o.orderkey=10 limit 2") + .validates(plan -> assertNodePresentInPlan(plan, DistinctLimitNode.class)); + } +} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestRedundantDistinctRemoval.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestRedundantDistinctRemoval.java new file mode 100644 index 0000000000000..6cbdb7c1ba8f3 --- /dev/null +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestRedundantDistinctRemoval.java @@ -0,0 +1,171 @@ +/* + * 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 com.facebook.presto.sql.planner.iterative.rule; + +import com.facebook.presto.spi.plan.AggregationNode; +import com.facebook.presto.spi.plan.TableScanNode; +import com.facebook.presto.spi.plan.ValuesNode; +import com.facebook.presto.spi.relation.VariableReferenceExpression; +import com.facebook.presto.sql.planner.TestTableConstraintsConnectorFactory; +import com.facebook.presto.sql.planner.iterative.properties.LogicalPropertiesProviderImpl; +import com.facebook.presto.sql.planner.iterative.rule.test.BaseRuleTest; +import com.facebook.presto.sql.planner.iterative.rule.test.RuleTester; +import com.facebook.presto.sql.relational.FunctionResolution; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import java.util.Optional; + +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.any; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.assignUniqueId; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.node; +import static java.util.Collections.emptyList; + +public class TestRedundantDistinctRemoval + extends BaseRuleTest +{ + private LogicalPropertiesProviderImpl logicalPropertiesProvider; + + @BeforeClass + public final void setUp() + { + tester = new RuleTester(emptyList(), ImmutableMap.of("exploit_constraints", Boolean.toString(true)), Optional.of(1), new TestTableConstraintsConnectorFactory(1)); + logicalPropertiesProvider = new LogicalPropertiesProviderImpl(new FunctionResolution(tester.getMetadata().getFunctionAndTypeManager())); + } + + @Test + public void singleTableTests() + { + tester().assertThat(ImmutableSet.of(new RemoveRedundantDistinct()), logicalPropertiesProvider) + .on("SELECT DISTINCT acctbal FROM customer WHERE custkey = 100") + .validates(plan -> assertNodeRemovedFromPlan(plan, AggregationNode.class)); + + tester().assertThat(ImmutableSet.of(new RemoveRedundantDistinct()), logicalPropertiesProvider) + .on("SELECT DISTINCT orderkey, custkey FROM orders") + .validates(plan -> assertNodeRemovedFromPlan(plan, AggregationNode.class)); + + tester().assertThat(ImmutableSet.of(new RemoveRedundantDistinct()), logicalPropertiesProvider) + .on("SELECT DISTINCT * FROM (VALUES (1,2)) t1 (c1, c2)") + .matches(any(node(ValuesNode.class))); + + tester().assertThat(ImmutableSet.of(new RemoveRedundantDistinct()), logicalPropertiesProvider) + .on("SELECT DISTINCT orderkey, custkey FROM orders") + .matches(any(node(TableScanNode.class))); + + tester().assertThat(new RemoveRedundantDistinct(), logicalPropertiesProvider) + .on(p -> { + VariableReferenceExpression c = p.variable("c"); + VariableReferenceExpression unique = p.variable("unique"); + return p.aggregation(builder -> builder + .singleGroupingSet(c, unique) + .source(p.assignUniqueId(unique, + p.values(5, c)))); + }) + .matches(assignUniqueId("unique", + node(ValuesNode.class))); + + // Negative test cases + tester().assertThat(ImmutableSet.of(new RemoveRedundantDistinct()), logicalPropertiesProvider) + .on("SELECT DISTINCT custkey FROM orders") + .matches(any(node(AggregationNode.class, (node(TableScanNode.class))))); + + tester().assertThat(ImmutableSet.of(new RemoveRedundantDistinct()), logicalPropertiesProvider) + .on("SELECT DISTINCT custkey+5 FROM customer") + .validates(plan -> assertNodePresentInPlan(plan, AggregationNode.class)); + + tester().assertThat(ImmutableSet.of(new RemoveRedundantDistinct()), logicalPropertiesProvider) + .on("SELECT DISTINCT acctbal FROM customer WHERE custkey + 5= 100") + .validates(plan -> assertNodePresentInPlan(plan, AggregationNode.class)); + + tester().assertThat(ImmutableSet.of(new RemoveRedundantDistinct()), logicalPropertiesProvider) + .on("SELECT DISTINCT totalprice from orders where Orderkey in (1,2,3,3)") + .validates(plan -> assertNodePresentInPlan(plan, AggregationNode.class)); + + tester().assertThat(ImmutableSet.of(new RemoveRedundantDistinct()), logicalPropertiesProvider) + .on("SELECT DISTINCT totalprice FROM orders WHERE orderkey = (SELECT orderkey from lineitem limit 1)") + .validates(plan -> assertNodePresentInPlan(plan, AggregationNode.class)); + } + + //TODO add some outer join tests, including negative tests where keys should not propagate + + @Test + public void testRedundantDistinctRemovalOneToNJoin() + { + tester().assertThat(ImmutableSet.of(new RemoveRedundantDistinct()), logicalPropertiesProvider) + .on("select distinct * from orders o inner join customer c on o.custkey = c.custkey") + .validates(plan -> assertNodeRemovedFromPlan(plan, AggregationNode.class)); + tester().assertThat(ImmutableSet.of(new RemoveRedundantDistinct()), logicalPropertiesProvider) + .on("select distinct l.orderkey from orders o inner join customer c on o.custkey = c.custkey inner join lineitem l on o.orderkey = l.orderkey where o.orderkey = 10 and l.linenumber = 10") + .validates(plan -> assertNodeRemovedFromPlan(plan, AggregationNode.class)); + tester().assertThat(ImmutableSet.of(new RemoveRedundantDistinct()), logicalPropertiesProvider) + .on("select distinct o.orderkey from orders o inner join customer c on o.custkey = c.custkey inner join lineitem l on o.orderkey = l.orderkey where o.orderkey = 10 and l.linenumber = 10") + .validates(plan -> assertNodeRemovedFromPlan(plan, AggregationNode.class)); + tester().assertThat(ImmutableSet.of(new RemoveRedundantDistinct()), logicalPropertiesProvider) + .on("select distinct c.custkey from orders o inner join customer c on o.custkey = c.custkey inner join lineitem l on o.orderkey = l.orderkey where o.orderkey = 10 and l.linenumber = 10") + .validates(plan -> assertNodeRemovedFromPlan(plan, AggregationNode.class)); + tester().assertThat(ImmutableSet.of(new RemoveRedundantDistinct()), logicalPropertiesProvider) + .on("SELECT distinct orderkey FROM orders WHERE orderkey IN (SELECT orderkey FROM lineitem WHERE linenumber in (1,2,3,4))") + .validates(plan -> assertNodeRemovedFromPlan(plan, AggregationNode.class)); + } + + @Test + public void testRedundantDistinctRemovalNtoMJoin() + { + tester().assertThat(ImmutableSet.of(new RemoveRedundantDistinct()), logicalPropertiesProvider) + .on("select distinct o.orderkey, c.custkey from orders o inner join customer c on o.custkey > c.custkey") + .validates(plan -> assertNodeRemovedFromPlan(plan, AggregationNode.class)); + } + + @Test + public void testRedundantDistinctRemovalOneToOneJoin() + { + tester().assertThat(ImmutableSet.of(new RemoveRedundantDistinct()), logicalPropertiesProvider) + .on("select distinct o.orderkey, l.linenumber from orders o inner join lineitem l on o.orderkey = l.orderkey where l.linenumber = 10") + .validates(plan -> assertNodeRemovedFromPlan(plan, AggregationNode.class)); + tester().assertThat(ImmutableSet.of(new RemoveRedundantDistinct()), logicalPropertiesProvider) + .on("select distinct l.orderkey, l.linenumber from orders o inner join lineitem l on o.orderkey = l.orderkey where l.linenumber = 10") + .validates(plan -> assertNodeRemovedFromPlan(plan, AggregationNode.class)); + tester().assertThat(ImmutableSet.of(new RemoveRedundantDistinct()), logicalPropertiesProvider) + .on("SELECT DISTINCT o.orderkey FROM orders o LEFT JOIN customer c ON o.custkey = c.custkey") + .validates(plan -> assertNodeRemovedFromPlan(plan, AggregationNode.class)); + } + + @Test + public void doesNotFire() + { + tester().assertThat(new RemoveRedundantDistinct(), logicalPropertiesProvider) + .on(p -> { + VariableReferenceExpression c = p.variable("c"); + return p.aggregation(builder -> builder + .singleGroupingSet(c) + .source(p.limit( + 5, + p.values(5, c)))); + }) + .doesNotFire(); + } + + @Test + public void testFeatureDisabled() + { + // Disable the feature and verify that optimization rule is not applied. + RuleTester newTester = new RuleTester(emptyList(), ImmutableMap.of("exploit_constraints", Boolean.toString(false))); + + newTester.assertThat(ImmutableSet.of(new RemoveRedundantDistinct()), logicalPropertiesProvider) + .on("select distinct o.orderkey, l.linenumber from orders o inner join lineitem l on o.orderkey = l.orderkey where l.linenumber = 10") + .validates(plan -> assertNodePresentInPlan(plan, AggregationNode.class)); + } +} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestRedundantLimitRemoval.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestRedundantLimitRemoval.java new file mode 100644 index 0000000000000..61bd4a6bb4128 --- /dev/null +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestRedundantLimitRemoval.java @@ -0,0 +1,136 @@ +/* + * 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 com.facebook.presto.sql.planner.iterative.rule; + +import com.facebook.presto.spi.plan.AggregationNode; +import com.facebook.presto.spi.plan.LimitNode; +import com.facebook.presto.spi.plan.ValuesNode; +import com.facebook.presto.spi.relation.VariableReferenceExpression; +import com.facebook.presto.sql.planner.TestTableConstraintsConnectorFactory; +import com.facebook.presto.sql.planner.iterative.properties.LogicalPropertiesProviderImpl; +import com.facebook.presto.sql.planner.iterative.rule.test.BaseRuleTest; +import com.facebook.presto.sql.planner.iterative.rule.test.RuleTester; +import com.facebook.presto.sql.relational.FunctionResolution; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import java.util.Optional; + +import static com.facebook.presto.common.type.BigintType.BIGINT; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.node; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.output; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.values; +import static com.facebook.presto.sql.planner.iterative.rule.test.PlanBuilder.expression; +import static java.util.Collections.emptyList; + +public class TestRedundantLimitRemoval + extends BaseRuleTest +{ + private LogicalPropertiesProviderImpl logicalPropertiesProvider; + + @BeforeClass + public final void setUp() + { + tester = new RuleTester(emptyList(), ImmutableMap.of("exploit_constraints", Boolean.toString(true)), Optional.of(1), new TestTableConstraintsConnectorFactory(1)); + logicalPropertiesProvider = new LogicalPropertiesProviderImpl(new FunctionResolution(tester.getMetadata().getFunctionAndTypeManager())); + } + + @Test + public void singleTableTests() + { + tester().assertThat(new RemoveRedundantLimit(), logicalPropertiesProvider) + .on(p -> + p.limit( + 10, + p.aggregation(builder -> builder + .addAggregation(p.variable("c"), expression("count(foo)"), ImmutableList.of(BIGINT)) + .globalGrouping() + .source(p.values(p.variable("foo")))))) + .matches( + node(AggregationNode.class, + node(ValuesNode.class))); + + //remove limit with ties + tester().assertThat(new RemoveRedundantLimit(), logicalPropertiesProvider) + .on(p -> { + VariableReferenceExpression c = p.variable("c"); + return p.limit( + 10, + p.values(5, c)); + }) + .matches(values("c")); + + tester().assertThat(ImmutableSet.of(new RemoveRedundantLimit()), logicalPropertiesProvider) + .on("SELECT count(*) FROM orders LIMIT 2") + .validates(plan -> assertNodeRemovedFromPlan(plan, LimitNode.class)); + + tester().assertThat(ImmutableSet.of(new RemoveRedundantLimit()), logicalPropertiesProvider) + .on("SELECT * FROM (VALUES 1,2,3,4,5,6) AS t1 LIMIT 10") + .matches(output( + values(ImmutableList.of("x")))); + + tester().assertThat(ImmutableSet.of(new RemoveRedundantLimit()), logicalPropertiesProvider) + .on("select a from orders inner join (values(2)) t(a) ON orderkey=1 limit 3") + .validates(plan -> assertNodeRemovedFromPlan(plan, LimitNode.class)); + + //negative tests + tester().assertThat(ImmutableSet.of(new RemoveRedundantLimit()), logicalPropertiesProvider) + .on("SELECT orderkey, count(*) FROM orders GROUP BY orderkey ORDER BY 1 LIMIT 10") + .validates(plan -> assertNodePresentInPlan(plan, LimitNode.class)); + tester().assertThat(ImmutableSet.of(new RemoveRedundantLimit()), logicalPropertiesProvider) + .on("select a from orders left join (values(2)) t(a) ON orderkey=1 limit 3") + .validates(plan -> assertNodePresentInPlan(plan, LimitNode.class)); + } + + @Test + public void complexQueryTests() + { + //TODO more join, complex query tests + tester().assertThat(ImmutableSet.of(new RemoveRedundantLimit()), logicalPropertiesProvider) + .on("select totalprice from orders o inner join customer c on o.custkey = c.custkey where o.orderkey=10 limit 2") + .validates(plan -> assertNodeRemovedFromPlan(plan, LimitNode.class)); + + //negative tests + tester().assertThat(ImmutableSet.of(new RemoveRedundantLimit()), logicalPropertiesProvider) + .on("select totalprice from orders o inner join customer c on o.custkey = c.custkey where o.orderkey>10 limit 2") + .validates(plan -> assertNodePresentInPlan(plan, LimitNode.class)); + } + + @Test + public void doesNotFire() + { + tester().assertThat(new RemoveRedundantLimit(), logicalPropertiesProvider) + .on(p -> { + VariableReferenceExpression c = p.variable("c"); + return p.limit( + 10, + p.values(12, c)); + }) + .doesNotFire(); + } + + @Test + public void testFeatureDisabled() + { + // Disable the feature and verify that optimization rule is not applied. + RuleTester newTester = new RuleTester(emptyList(), ImmutableMap.of("exploit_constraints", Boolean.toString(false))); + + newTester.assertThat(ImmutableSet.of(new RemoveRedundantLimit()), logicalPropertiesProvider) + .on("select totalprice from orders o inner join customer c on o.custkey = c.custkey where o.orderkey=10 limit 2") + .validates(plan -> assertNodePresentInPlan(plan, LimitNode.class)); + } +} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestRedundantSortRemoval.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestRedundantSortRemoval.java new file mode 100644 index 0000000000000..dec0caea07adb --- /dev/null +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestRedundantSortRemoval.java @@ -0,0 +1,139 @@ +/* + * 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 com.facebook.presto.sql.planner.iterative.rule; + +import com.facebook.presto.spi.plan.AggregationNode; +import com.facebook.presto.spi.plan.ValuesNode; +import com.facebook.presto.sql.planner.TestTableConstraintsConnectorFactory; +import com.facebook.presto.sql.planner.iterative.properties.LogicalPropertiesProviderImpl; +import com.facebook.presto.sql.planner.iterative.rule.test.BaseRuleTest; +import com.facebook.presto.sql.planner.iterative.rule.test.RuleTester; +import com.facebook.presto.sql.planner.plan.SortNode; +import com.facebook.presto.sql.relational.FunctionResolution; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import java.util.Optional; + +import static com.facebook.presto.common.type.BigintType.BIGINT; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.node; +import static com.facebook.presto.sql.planner.iterative.rule.test.PlanBuilder.expression; +import static java.util.Collections.emptyList; + +public class TestRedundantSortRemoval + extends BaseRuleTest +{ + private LogicalPropertiesProviderImpl logicalPropertiesProvider; + + @BeforeClass + public final void setUp() + { + tester = new RuleTester(emptyList(), ImmutableMap.of("exploit_constraints", Boolean.toString(true)), Optional.of(1), new TestTableConstraintsConnectorFactory(1)); + logicalPropertiesProvider = new LogicalPropertiesProviderImpl(new FunctionResolution(tester.getMetadata().getFunctionAndTypeManager())); + } + + @Test + public void singleTableTests() + { + //sorting one record result of a single group aggregate + tester().assertThat(new RemoveRedundantSort(), logicalPropertiesProvider) + .on(p -> + p.sort( + ImmutableList.of(p.variable("c")), + p.aggregation(builder -> builder + .addAggregation(p.variable("c"), expression("count(foo)"), ImmutableList.of(BIGINT)) + .globalGrouping() + .source(p.values(p.variable("foo")))))) + .matches( + node(AggregationNode.class, + node(ValuesNode.class))); + + tester().assertThat(ImmutableSet.of(new RemoveRedundantSort()), logicalPropertiesProvider) + .on("select max(totalprice) from orders order by 1") + .validates(plan -> assertNodeRemovedFromPlan(plan, SortNode.class)); + + tester().assertThat(ImmutableSet.of(new RemoveRedundantSort()), logicalPropertiesProvider) + .on("select count(*), totalprice from orders where totalprice = 101.10 group by totalprice order by 1, 2") + .validates(plan -> assertNodeRemovedFromPlan(plan, SortNode.class)); + + //zero cardinality + tester().assertThat(new RemoveRedundantSort(), logicalPropertiesProvider) + .on(p -> + p.sort( + ImmutableList.of(p.variable("c")), + p.values(p.variable("foo")))) + .matches(node(ValuesNode.class)); + + //binding a primary key + tester().assertThat(ImmutableSet.of(new RemoveRedundantSort()), logicalPropertiesProvider) + .on("SELECT totalprice FROM orders WHERE orderkey = 10 ORDER BY totalprice") + .validates(plan -> assertNodeRemovedFromPlan(plan, SortNode.class)); + + tester().assertThat(ImmutableSet.of(new RemoveRedundantSort()), logicalPropertiesProvider) + .on("SELECT quantity FROM lineitem WHERE orderkey = 10 and linenumber = 100 ORDER BY quantity") + .validates(plan -> assertNodeRemovedFromPlan(plan, SortNode.class)); + + // Negative test cases + //TODO add more negative tests, i.e. operators that do not propagate keys like + tester().assertThat(ImmutableSet.of(new RemoveRedundantSort()), logicalPropertiesProvider) + .on("SELECT quantity FROM lineitem WHERE orderkey = 10 ORDER BY quantity") + .validates(plan -> assertNodePresentInPlan(plan, SortNode.class)); + + tester().assertThat(ImmutableSet.of(), logicalPropertiesProvider) + .on("select max(totalprice) from orders order by 1") + .validates(plan -> assertNodePresentInPlan(plan, SortNode.class)); + } + + @Test + public void complexQueryTests() + { + //TODO more join, complex query tests + tester().assertThat(ImmutableSet.of(new RemoveRedundantSort()), logicalPropertiesProvider) + .on("select totalprice from orders o inner join customer c on o.custkey = c.custkey where o.orderkey=10 order by totalprice") + .validates(plan -> assertNodeRemovedFromPlan(plan, SortNode.class)); + + //negative tests + tester().assertThat(ImmutableSet.of(new RemoveRedundantSort()), logicalPropertiesProvider) + .on("select totalprice from orders o inner join customer c on o.custkey = c.custkey where o.orderkey>10 order by totalprice") + .validates(plan -> assertNodePresentInPlan(plan, SortNode.class)); + } + + @Test + public void doesNotFire() + { + tester().assertThat(new RemoveRedundantSort(), logicalPropertiesProvider) + .on(p -> + p.sort( + ImmutableList.of(p.variable("c")), + p.aggregation(builder -> builder + .addAggregation(p.variable("c"), expression("count(foo)"), ImmutableList.of(BIGINT)) + .singleGroupingSet(p.variable("foo")) + .source(p.values(20, p.variable("foo")))))) + .doesNotFire(); + } + + @Test + public void testFeatureDisabled() + { + // Disable the feature and verify that optimization rule is not applied. + RuleTester newTester = new RuleTester(emptyList(), ImmutableMap.of("exploit_constraints", Boolean.toString(false))); + + newTester.assertThat(ImmutableSet.of(new RemoveRedundantSort()), logicalPropertiesProvider) + .on("select max(totalprice) from orders order by 1") + .validates(plan -> assertNodePresentInPlan(plan, SortNode.class)); + } +} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestRedundantTopNRemoval.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestRedundantTopNRemoval.java new file mode 100644 index 0000000000000..a6d53d1aa1b0b --- /dev/null +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestRedundantTopNRemoval.java @@ -0,0 +1,134 @@ +/* + * 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 com.facebook.presto.sql.planner.iterative.rule; + +import com.facebook.presto.spi.plan.AggregationNode; +import com.facebook.presto.spi.plan.TopNNode; +import com.facebook.presto.spi.plan.ValuesNode; +import com.facebook.presto.sql.planner.TestTableConstraintsConnectorFactory; +import com.facebook.presto.sql.planner.iterative.properties.LogicalPropertiesProviderImpl; +import com.facebook.presto.sql.planner.iterative.rule.test.BaseRuleTest; +import com.facebook.presto.sql.planner.iterative.rule.test.RuleTester; +import com.facebook.presto.sql.relational.FunctionResolution; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import java.util.Optional; + +import static com.facebook.presto.common.type.BigintType.BIGINT; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.anyTree; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.node; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.output; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.tableScan; +import static com.facebook.presto.sql.planner.iterative.rule.test.PlanBuilder.expression; +import static java.util.Collections.emptyList; + +public class TestRedundantTopNRemoval + extends BaseRuleTest +{ + private LogicalPropertiesProviderImpl logicalPropertiesProvider; + + @BeforeClass + public final void setUp() + { + tester = new RuleTester(emptyList(), ImmutableMap.of("exploit_constraints", Boolean.toString(true)), Optional.of(1), new TestTableConstraintsConnectorFactory(1)); + logicalPropertiesProvider = new LogicalPropertiesProviderImpl(new FunctionResolution(tester.getMetadata().getFunctionAndTypeManager())); + } + + @Test + public void singleTableTests() + { + tester().assertThat(ImmutableSet.of(new MergeLimitWithSort(), new RemoveRedundantTopN()), logicalPropertiesProvider) + .on("SELECT totalprice FROM orders WHERE orderkey = 10 ORDER BY totalprice LIMIT 10") + .validates(plan -> assertNodeRemovedFromPlan(plan, TopNNode.class)); + + //single group and limit is 10 + tester().assertThat(new RemoveRedundantTopN(), logicalPropertiesProvider) + .on(p -> + p.topN( + 10, + ImmutableList.of(p.variable("c")), + p.aggregation(builder -> builder + .addAggregation(p.variable("c"), expression("count(foo)"), ImmutableList.of(BIGINT)) + .globalGrouping() + .source(p.values(p.variable("foo")))))) + .matches( + node(AggregationNode.class, + node(ValuesNode.class))); + + tester().assertThat(ImmutableSet.of(new MergeLimitWithSort(), new RemoveRedundantTopN()), logicalPropertiesProvider) + .on("SELECT count(*) FROM orders ORDER BY 1 LIMIT 10") + .validates(plan -> assertNodeRemovedFromPlan(plan, TopNNode.class)); + + //negative tests + tester().assertThat(ImmutableSet.of(new MergeLimitWithSort()), logicalPropertiesProvider) + .on("SELECT totalprice FROM orders WHERE orderkey = 10 ORDER BY totalprice LIMIT 10") + .validates(plan -> assertNodePresentInPlan(plan, TopNNode.class)); + + tester().assertThat(ImmutableSet.of(new MergeLimitWithSort(), new RemoveRedundantTopN()), logicalPropertiesProvider) + .on("SELECT orderkey, count(*) FROM orders GROUP BY orderkey ORDER BY 1 LIMIT 10") + .matches(output( + node(TopNNode.class, + anyTree( + tableScan("orders"))))); + } + + @Test + public void complexQueryTests() + { + tester().assertThat(ImmutableSet.of(new MergeLimitWithSort(), new RemoveRedundantTopN()), logicalPropertiesProvider) + .on("select totalprice from orders o inner join customer c on o.custkey = c.custkey where o.orderkey=10 order by totalprice limit 10") + .validates(plan -> assertNodeRemovedFromPlan(plan, TopNNode.class)); + tester().assertThat(ImmutableSet.of(new MergeLimitWithSort(), new RemoveRedundantTopN()), logicalPropertiesProvider) + .on("select a from orders join (values(2)) t(a) ON orderkey=1 order by orderkey limit 3") + .validates(plan -> assertNodeRemovedFromPlan(plan, TopNNode.class)); + + //negative test + tester().assertThat(ImmutableSet.of(new MergeLimitWithSort()), logicalPropertiesProvider) + .on("select totalprice from orders o inner join customer c on o.custkey = c.custkey where o.orderkey=10 order by totalprice limit 10") + .validates(plan -> assertNodePresentInPlan(plan, TopNNode.class)); + tester().assertThat(ImmutableSet.of(new MergeLimitWithSort(), new RemoveRedundantTopN()), logicalPropertiesProvider) + .on("select a from orders left join (values(2)) t(a) ON orderkey=1 order by orderkey limit 3") + .validates(plan -> assertNodePresentInPlan(plan, TopNNode.class)); + } + + @Test + public void doesNotFire() + { + tester().assertThat(new RemoveRedundantTopN(), logicalPropertiesProvider) + .on(p -> + p.topN( + 10, + ImmutableList.of(p.variable("c")), + p.aggregation(builder -> builder + .addAggregation(p.variable("c"), expression("count(foo)"), ImmutableList.of(BIGINT)) + .singleGroupingSet(p.variable("foo")) + .source(p.values(20, p.variable("foo")))))) + .doesNotFire(); + } + + @Test + public void testFeatureDisabled() + { + // Disable the feature and verify that optimization rule is not applied. + RuleTester newTester = new RuleTester(emptyList(), ImmutableMap.of("exploit_constraints", Boolean.toString(false))); + + newTester.assertThat(ImmutableSet.of(new MergeLimitWithSort(), new RemoveRedundantTopN()), logicalPropertiesProvider) + .on("SELECT totalprice FROM orders WHERE orderkey = 10 ORDER BY totalprice LIMIT 10") + .validates(plan -> assertNodePresentInPlan(plan, TopNNode.class)); + } +} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/BaseRuleTest.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/BaseRuleTest.java index 95ba4df4d45bb..04ce733e96aab 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/BaseRuleTest.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/BaseRuleTest.java @@ -16,6 +16,7 @@ import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.spi.Plugin; +import com.facebook.presto.sql.planner.Plan; import com.google.common.collect.ImmutableList; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; @@ -23,10 +24,14 @@ import java.util.List; import static com.facebook.airlift.testing.Closeables.closeAllRuntimeException; +import static com.facebook.presto.sql.planner.optimizations.PlanNodeSearcher.searchFrom; +import static com.facebook.presto.util.MorePredicates.isInstanceOfAny; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; public abstract class BaseRuleTest { - private RuleTester tester; + protected RuleTester tester; private List plugins; public BaseRuleTest(Plugin... plugins) @@ -35,7 +40,7 @@ public BaseRuleTest(Plugin... plugins) } @BeforeClass - public final void setUp() + public void setUp() { tester = new RuleTester(plugins); } @@ -61,4 +66,22 @@ protected FunctionAndTypeManager getFunctionManager() { return tester.getMetadata().getFunctionAndTypeManager(); } + + protected void assertNodeRemovedFromPlan(Plan plan, Class nodeClass) + { + assertFalse( + searchFrom(plan.getRoot()) + .where(isInstanceOfAny(nodeClass)) + .matches(), + "Unexpected " + nodeClass.toString() + " in plan after optimization. "); + } + + protected void assertNodePresentInPlan(Plan plan, Class nodeClass) + { + assertTrue( + searchFrom(plan.getRoot()) + .where(isInstanceOfAny(nodeClass)) + .matches(), + "Expected " + nodeClass.toString() + " in plan after optimization. "); + } } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/PlanBuilder.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/PlanBuilder.java index 9c7b995acea7b..04e6470f37e3f 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/PlanBuilder.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/PlanBuilder.java @@ -25,11 +25,13 @@ import com.facebook.presto.spi.SchemaTableName; import com.facebook.presto.spi.TableHandle; import com.facebook.presto.spi.WarningCollector; +import com.facebook.presto.spi.constraints.TableConstraint; import com.facebook.presto.spi.function.FunctionHandle; import com.facebook.presto.spi.plan.AggregationNode; import com.facebook.presto.spi.plan.AggregationNode.Aggregation; import com.facebook.presto.spi.plan.AggregationNode.Step; import com.facebook.presto.spi.plan.Assignments; +import com.facebook.presto.spi.plan.DistinctLimitNode; import com.facebook.presto.spi.plan.ExceptNode; import com.facebook.presto.spi.plan.FilterNode; import com.facebook.presto.spi.plan.IntersectNode; @@ -100,6 +102,7 @@ import java.util.function.Consumer; import java.util.stream.Stream; +import static com.facebook.presto.common.block.SortOrder.ASC_NULLS_FIRST; import static com.facebook.presto.common.type.BigintType.BIGINT; import static com.facebook.presto.common.type.UnknownType.UNKNOWN; import static com.facebook.presto.common.type.VarbinaryType.VARBINARY; @@ -269,10 +272,22 @@ public TopNNode topN(long count, List orderBy, Plan idAllocator.getNextId(), source, count, - new OrderingScheme(orderBy.stream().map(variable -> new Ordering(variable, SortOrder.ASC_NULLS_FIRST)).collect(toImmutableList())), + new OrderingScheme(orderBy.stream().map(variable -> new Ordering(variable, ASC_NULLS_FIRST)).collect(toImmutableList())), TopNNode.Step.SINGLE); } + public DistinctLimitNode distinctLimit(long count, List distinctSymbols, PlanNode source) + { + return new DistinctLimitNode( + source.getSourceLocation(), + idAllocator.getNextId(), + source, + count, + false, + distinctSymbols, + Optional.empty()); + } + public SampleNode sample(double sampleRatio, SampleNode.Type type, PlanNode source) { return new SampleNode(source.getSourceLocation(), idAllocator.getNextId(), source, sampleRatio, type); @@ -515,6 +530,26 @@ public TableScanNode tableScan( tableHandle, variables, assignments, + ImmutableList.of(), + currentConstraint, + enforcedConstraint); + } + + public TableScanNode tableScan( + TableHandle tableHandle, + List variables, + Map assignments, + TupleDomain currentConstraint, + TupleDomain enforcedConstraint, + List> tableConstraints) + { + return new TableScanNode( + Optional.empty(), + idAllocator.getNextId(), + tableHandle, + variables, + assignments, + tableConstraints, currentConstraint, enforcedConstraint); } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/RuleAssert.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/RuleAssert.java index f303ce8506309..029876dfa6a98 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/RuleAssert.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/RuleAssert.java @@ -26,6 +26,8 @@ import com.facebook.presto.metadata.Metadata; import com.facebook.presto.security.AccessControl; import com.facebook.presto.spi.WarningCollector; +import com.facebook.presto.spi.plan.LogicalProperties; +import com.facebook.presto.spi.plan.LogicalPropertiesProvider; import com.facebook.presto.spi.plan.PlanNode; import com.facebook.presto.spi.plan.PlanNodeId; import com.facebook.presto.spi.plan.PlanNodeIdAllocator; @@ -40,6 +42,7 @@ import com.facebook.presto.sql.planner.iterative.Memo; import com.facebook.presto.sql.planner.iterative.PlanNodeMatcher; import com.facebook.presto.sql.planner.iterative.Rule; +import com.facebook.presto.sql.planner.iterative.properties.LogicalPropertiesImpl; import com.facebook.presto.sql.planner.iterative.rule.TranslateExpressions; import com.facebook.presto.transaction.TransactionManager; import com.google.common.collect.ImmutableSet; @@ -70,8 +73,15 @@ public class RuleAssert private Session session; private TypeProvider types; private PlanNode plan; + private Optional logicalPropertiesProvider; public RuleAssert(Metadata metadata, StatsCalculator statsCalculator, CostCalculator costCalculator, Session session, Rule rule, TransactionManager transactionManager, AccessControl accessControl) + { + this(metadata, statsCalculator, costCalculator, session, rule, transactionManager, accessControl, Optional.empty()); + } + + public RuleAssert(Metadata metadata, StatsCalculator statsCalculator, CostCalculator costCalculator, Session session, Rule rule, + TransactionManager transactionManager, AccessControl accessControl, Optional logicalPropertiesProvider) { this.metadata = requireNonNull(metadata, "metadata is null"); this.statsCalculator = new TestingStatsCalculator(requireNonNull(statsCalculator, "statsCalculator is null")); @@ -80,6 +90,7 @@ public RuleAssert(Metadata metadata, StatsCalculator statsCalculator, CostCalcul this.rule = requireNonNull(rule, "rule is null"); this.transactionManager = requireNonNull(transactionManager, "transactionManager is null"); this.accessControl = requireNonNull(accessControl, "accessControl is null"); + this.logicalPropertiesProvider = requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider is null"); } public RuleAssert setSystemProperty(String key, String value) @@ -175,18 +186,42 @@ public void matches(PlanMatchPattern pattern) }); } + public void matches(LogicalProperties expectedLogicalProperties) + { + RuleApplication ruleApplication = applyRule(); + TypeProvider types = ruleApplication.types; + + if (!ruleApplication.wasRuleApplied()) { + fail(String.format( + "%s did not fire for:\n%s", + rule.getClass().getName(), + formatPlan(plan, types))); + } + + // ensure that the logical properties of the root group are equivalent to the expected logical properties + LogicalProperties rootNodeLogicalProperties = ruleApplication.getMemo().getLogicalProperties(ruleApplication.getMemo().getRootGroup()).get(); + if (!((LogicalPropertiesImpl) rootNodeLogicalProperties).equals((LogicalPropertiesImpl) expectedLogicalProperties)) { + fail(String.format( + "Logical properties of root node doesn't match expected logical properties\n" + + "\texpected: %s\n" + + "\tactual: %s", + expectedLogicalProperties, + rootNodeLogicalProperties)); + } + } + private RuleApplication applyRule() { PlanVariableAllocator variableAllocator = new PlanVariableAllocator(types.allVariables()); - Memo memo = new Memo(idAllocator, plan); + Memo memo = new Memo(idAllocator, plan, logicalPropertiesProvider); Lookup lookup = Lookup.from(planNode -> Stream.of(memo.resolve(planNode))); PlanNode memoRoot = memo.getNode(memo.getRootGroup()); - return inTransaction(session -> applyRule(rule, memoRoot, ruleContext(statsCalculator, costCalculator, variableAllocator, memo, lookup, session))); + return inTransaction(session -> applyRule(rule, memoRoot, ruleContext(statsCalculator, costCalculator, variableAllocator, memo, lookup, session), memo)); } - private static RuleApplication applyRule(Rule rule, PlanNode planNode, Rule.Context context) + private static RuleApplication applyRule(Rule rule, PlanNode planNode, Rule.Context context, Memo memo) { PlanNodeMatcher matcher = new PlanNodeMatcher(context.getLookup()); Match match = matcher.match(rule.getPattern(), planNode); @@ -199,7 +234,7 @@ private static RuleApplication applyRule(Rule rule, PlanNode planNode, Ru result = rule.apply(match.value(), match.captures(), context); } - return new RuleApplication(context.getLookup(), context.getStatsProvider(), context.getVariableAllocator().getTypes(), result); + return new RuleApplication(context.getLookup(), context.getStatsProvider(), context.getVariableAllocator().getTypes(), memo, result); } private String formatPlan(PlanNode plan, TypeProvider types) @@ -286,13 +321,15 @@ private static class RuleApplication private final StatsProvider statsProvider; private final TypeProvider types; private final Rule.Result result; + private final Memo memo; - public RuleApplication(Lookup lookup, StatsProvider statsProvider, TypeProvider types, Rule.Result result) + public RuleApplication(Lookup lookup, StatsProvider statsProvider, TypeProvider types, Memo memo, Rule.Result result) { this.lookup = requireNonNull(lookup, "lookup is null"); this.statsProvider = requireNonNull(statsProvider, "statsProvider is null"); this.types = requireNonNull(types, "types is null"); this.result = requireNonNull(result, "result is null"); + this.memo = requireNonNull(memo, "memo is null"); } private boolean wasRuleApplied() @@ -304,6 +341,11 @@ public PlanNode getTransformedPlan() { return result.getTransformedPlan().orElseThrow(() -> new IllegalStateException("Rule did not produce transformed plan")); } + + private Memo getMemo() + { + return memo; + } } public static class TestingStatsCalculator diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/RuleTester.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/RuleTester.java index c441a6c471b9b..0252e559b942e 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/RuleTester.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/RuleTester.java @@ -16,12 +16,19 @@ import com.facebook.presto.Session; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.security.AccessControl; +import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.ConnectorId; import com.facebook.presto.spi.Plugin; +import com.facebook.presto.spi.TableHandle; +import com.facebook.presto.spi.connector.ConnectorFactory; +import com.facebook.presto.spi.constraints.TableConstraint; +import com.facebook.presto.spi.plan.LogicalPropertiesProvider; import com.facebook.presto.split.PageSourceManager; import com.facebook.presto.split.SplitManager; import com.facebook.presto.sql.parser.SqlParser; +import com.facebook.presto.sql.planner.RuleStatsRecorder; import com.facebook.presto.sql.planner.assertions.OptimizerAssert; +import com.facebook.presto.sql.planner.iterative.IterativeOptimizer; import com.facebook.presto.sql.planner.iterative.Rule; import com.facebook.presto.sql.planner.optimizations.PlanOptimizer; import com.facebook.presto.testing.LocalQueryRunner; @@ -33,6 +40,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import static com.facebook.presto.testing.TestingSession.testSessionBuilder; import static java.util.Collections.emptyList; @@ -68,6 +76,11 @@ public RuleTester(List plugins, Map sessionProperties) } public RuleTester(List plugins, Map sessionProperties, Optional nodeCountForStats) + { + this(plugins, sessionProperties, nodeCountForStats, new TpchConnectorFactory(1)); + } + + public RuleTester(List plugins, Map sessionProperties, Optional nodeCountForStats, ConnectorFactory connectorFactory) { Session.SessionBuilder sessionBuilder = testSessionBuilder() .setCatalog(CATALOG_ID) @@ -84,7 +97,7 @@ public RuleTester(List plugins, Map sessionProperties, O .map(nodeCount -> LocalQueryRunner.queryRunnerWithFakeNodeCountForStats(session, nodeCount)) .orElseGet(() -> new LocalQueryRunner(session)); queryRunner.createCatalog(session.getCatalog().get(), - new TpchConnectorFactory(1), + connectorFactory, ImmutableMap.of()); plugins.stream().forEach(queryRunner::installPlugin); @@ -101,9 +114,35 @@ public RuleAssert assertThat(Rule rule) return new RuleAssert(metadata, queryRunner.getStatsCalculator(), queryRunner.getEstimatedExchangesCostCalculator(), session, rule, transactionManager, accessControl); } + public RuleAssert assertThat(Rule rule, LogicalPropertiesProvider logicalPropertiesProvider) + { + return new RuleAssert(metadata, queryRunner.getStatsCalculator(), queryRunner.getEstimatedExchangesCostCalculator(), session, rule, transactionManager, accessControl, Optional.of(logicalPropertiesProvider)); + } + + public OptimizerAssert assertThat(Set> rules) + { + PlanOptimizer optimizer = new IterativeOptimizer( + new RuleStatsRecorder(), + queryRunner.getStatsCalculator(), + queryRunner.getCostCalculator(), + rules); + return new OptimizerAssert(metadata, queryRunner, queryRunner.getStatsCalculator(), session, optimizer, transactionManager, accessControl); + } + + public OptimizerAssert assertThat(Set> rules, LogicalPropertiesProvider logicalPropertiesProvider) + { + PlanOptimizer optimizer = new IterativeOptimizer( + new RuleStatsRecorder(), + queryRunner.getStatsCalculator(), + queryRunner.getCostCalculator(), + Optional.of(logicalPropertiesProvider), + rules); + return new OptimizerAssert(metadata, queryRunner, queryRunner.getStatsCalculator(), session, optimizer, transactionManager, accessControl); + } + public OptimizerAssert assertThat(PlanOptimizer optimizer) { - return new OptimizerAssert(metadata, queryRunner.getStatsCalculator(), session, optimizer, transactionManager, accessControl); + return new OptimizerAssert(metadata, queryRunner, queryRunner.getStatsCalculator(), session, optimizer, transactionManager, accessControl); } @Override @@ -136,4 +175,12 @@ public ConnectorId getCurrentConnectorId() { return queryRunner.inTransaction(transactionSession -> metadata.getCatalogHandle(transactionSession, session.getCatalog().get())).get(); } + + public List> getTableConstraints(TableHandle tableHandle) + { + return queryRunner.inTransaction(transactionSession -> { + metadata.getCatalogHandle(transactionSession, session.getCatalog().get()); + return metadata.getTableMetadata(transactionSession, tableHandle).getMetadata().getTableConstraints(); + }); + } } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestAddExchangesPlans.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestAddExchangesPlans.java index 3970eafea7c0e..885058c6fdd00 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestAddExchangesPlans.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestAddExchangesPlans.java @@ -314,6 +314,7 @@ public void testTopNRowNumberIsExactlyPartitioned() " COUNT(*)\n" + " FROM (\n" + " VALUES\n" + + " (1, 2),\n" + " (1, 2)\n" + " ) t (a, b)\n" + " GROUP BY\n" + diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestConnectorOptimization.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestConnectorOptimization.java index c04cbf6281bfc..374dff4aa2946 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestConnectorOptimization.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestConnectorOptimization.java @@ -336,6 +336,7 @@ public PlanNode visitFilter(FilterNode node, Void context) Optional.of(new TestConnectorTableLayoutHandle(node.getPredicate()))), tableScanNode.getOutputVariables(), tableScanNode.getAssignments(), + tableScanNode.getTableConstraints(), TupleDomain.all(), TupleDomain.all()); } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/query/TestSubqueries.java b/presto-main/src/test/java/com/facebook/presto/sql/query/TestSubqueries.java index 1a4a5b84d7ca1..86fbfd0eb326c 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/query/TestSubqueries.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/query/TestSubqueries.java @@ -125,7 +125,7 @@ public void testCorrelatedSubqueriesWithLimit() "SELECT (SELECT count(*) FROM (VALUES 1, 1, 3) t(a) WHERE t.a=t2.b LIMIT 1) FROM (VALUES 1) t2(b)", UNSUPPORTED_CORRELATED_SUBQUERY_ERROR_MSG); assertExistsRewrittenToAggregationBelowJoin( - "SELECT EXISTS(SELECT 1 FROM (values ('x', 1)) u(x, cid) WHERE x = 'x' AND t.cid = cid LIMIT 1) " + + "SELECT EXISTS(SELECT 1 FROM (values ('x', 1), ('y', 2)) u(x, cid) WHERE x = 'x' AND t.cid = cid LIMIT 1) " + "FROM (values 1) t(cid)", "VALUES true", false); diff --git a/presto-parquet/src/main/java/com/facebook/presto/parquet/rule/ParquetDereferencePushDown.java b/presto-parquet/src/main/java/com/facebook/presto/parquet/rule/ParquetDereferencePushDown.java index b18b76ee9d8fd..dcdb08146eb04 100644 --- a/presto-parquet/src/main/java/com/facebook/presto/parquet/rule/ParquetDereferencePushDown.java +++ b/presto-parquet/src/main/java/com/facebook/presto/parquet/rule/ParquetDereferencePushDown.java @@ -390,6 +390,7 @@ public PlanNode visitProject(ProjectNode project, Void context) tableScan.getTable(), newOutputVariables, newAssignments, + tableScan.getTableConstraints(), tableScan.getCurrentConstraint(), tableScan.getEnforcedConstraint()); diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/ConnectorSession.java b/presto-spi/src/main/java/com/facebook/presto/spi/ConnectorSession.java index 50627faf7368c..48d9ca3e960f6 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/ConnectorSession.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/ConnectorSession.java @@ -53,4 +53,9 @@ default String getUser() T getProperty(String name, Class type); Optional getSchema(); + + default boolean isReadConstraints() + { + return false; + } } diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/ConnectorTableMetadata.java b/presto-spi/src/main/java/com/facebook/presto/spi/ConnectorTableMetadata.java index bce4941af0c47..9b2a68cb5cece 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/ConnectorTableMetadata.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/ConnectorTableMetadata.java @@ -13,6 +13,8 @@ */ package com.facebook.presto.spi; +import com.facebook.presto.spi.constraints.TableConstraint; + import java.util.ArrayList; import java.util.Collections; import java.util.LinkedHashMap; @@ -20,6 +22,7 @@ import java.util.Map; import java.util.Optional; +import static java.util.Collections.emptyList; import static java.util.Collections.emptyMap; import static java.util.Objects.requireNonNull; @@ -29,6 +32,7 @@ public class ConnectorTableMetadata private final Optional comment; private final List columns; private final Map properties; + private final List> tableConstraints; public ConnectorTableMetadata(SchemaTableName table, List columns) { @@ -37,19 +41,40 @@ public ConnectorTableMetadata(SchemaTableName table, List column public ConnectorTableMetadata(SchemaTableName table, List columns, Map properties) { - this(table, columns, properties, Optional.empty()); + this(table, columns, properties, Optional.empty(), emptyList()); } public ConnectorTableMetadata(SchemaTableName table, List columns, Map properties, Optional comment) + { + this(table, columns, properties, comment, emptyList()); + } + + public ConnectorTableMetadata(SchemaTableName table, List columns, Map properties, Optional comment, List> tableConstraints) { requireNonNull(table, "table is null"); requireNonNull(columns, "columns is null"); requireNonNull(comment, "comment is null"); + requireNonNull(tableConstraints, "tableConstraints is null"); this.table = table; this.columns = Collections.unmodifiableList(new ArrayList<>(columns)); this.properties = Collections.unmodifiableMap(new LinkedHashMap<>(properties)); this.comment = comment; + this.tableConstraints = Collections.unmodifiableList(new ArrayList<>(tableConstraints)); + } + + //rebase a list ot table constraints of column reference type T on column reference type R + public static List> rebaseTableConstraints(List> tableConstraints, Map assignments) + { + List> mappedTableConstraints = new ArrayList<>(); + tableConstraints.stream().forEach(tableConstraint -> + { + Optional> mappedConstraint = tableConstraint.rebaseConstraint(assignments); + if (mappedConstraint.isPresent()) { + mappedTableConstraints.add(mappedConstraint.get()); + } + }); + return mappedTableConstraints; } public SchemaTableName getTable() @@ -72,12 +97,18 @@ public Optional getComment() return comment; } + public List> getTableConstraints() + { + return tableConstraints; + } + @Override public String toString() { StringBuilder sb = new StringBuilder("ConnectorTableMetadata{"); sb.append("table=").append(table); sb.append(", columns=").append(columns); + sb.append(", tableConstraints=").append(tableConstraints); sb.append(", properties=").append(properties); comment.ifPresent(value -> sb.append(", comment='").append(value).append("'")); sb.append('}'); diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/constraints/PrimaryKeyConstraint.java b/presto-spi/src/main/java/com/facebook/presto/spi/constraints/PrimaryKeyConstraint.java new file mode 100644 index 0000000000000..386f138c5cd40 --- /dev/null +++ b/presto-spi/src/main/java/com/facebook/presto/spi/constraints/PrimaryKeyConstraint.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 com.facebook.presto.spi.constraints; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +public class PrimaryKeyConstraint + extends UniqueConstraint +{ + @JsonCreator + public PrimaryKeyConstraint(@JsonProperty("name") String name, + @JsonProperty("columns")Set columnNames, + @JsonProperty("enforced") boolean enabled, + @JsonProperty("rely") boolean rely) + { + this(Optional.of(name), columnNames, enabled, rely); + } + + private PrimaryKeyConstraint(Optional name, Set columnNames, boolean enabled, boolean rely) + { + super(name, columnNames, enabled, rely); + } + + @Override + public Optional> rebaseConstraint(Map assignments) + { + if (this.getColumns().stream().allMatch(assignments::containsKey)) { + return Optional.of(new PrimaryKeyConstraint(this.getName(), + this.getColumns().stream().map(assignments::get).collect(Collectors.toSet()), + this.isEnforced(), + this.isRely())); + } + else { + return Optional.empty(); + } + } +} diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/constraints/TableConstraint.java b/presto-spi/src/main/java/com/facebook/presto/spi/constraints/TableConstraint.java new file mode 100644 index 0000000000000..851f2f301df79 --- /dev/null +++ b/presto-spi/src/main/java/com/facebook/presto/spi/constraints/TableConstraint.java @@ -0,0 +1,115 @@ +/* + * 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 com.facebook.presto.spi.constraints; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; + +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; + +import static java.util.Collections.unmodifiableSet; +import static java.util.Objects.requireNonNull; + +@JsonTypeInfo( + use = JsonTypeInfo.Id.NAME, + include = JsonTypeInfo.As.PROPERTY, + property = "@type") +@JsonSubTypes({ + @JsonSubTypes.Type(value = PrimaryKeyConstraint.class, name = "primarykeyconstraint"), + @JsonSubTypes.Type(value = UniqueConstraint.class, name = "uniqueconstraint")}) +public abstract class TableConstraint +{ + private final Optional name; + private final boolean enforced; + private final boolean rely; + private final Set columns; + + protected TableConstraint(Optional name, Set columnNames, boolean enforced, boolean rely) + { + this.enforced = requireNonNull(enforced, "enabled is null"); + this.rely = requireNonNull(rely, "rely is null"); + this.name = requireNonNull(name, "name is null"); + requireNonNull(columnNames, "columnNames is null"); + if (columnNames.isEmpty()) { + throw new IllegalArgumentException("columnNames is empty."); + } + this.columns = unmodifiableSet(new HashSet<>(columnNames)); + } + + public abstract Optional> rebaseConstraint(Map assignments); + + @JsonProperty + public Optional getName() + { + return name; + } + + @JsonProperty + public boolean isEnforced() + { + return enforced; + } + + @JsonProperty + public boolean isRely() + { + return rely; + } + + @JsonProperty + public Set getColumns() + { + return columns; + } + + @Override + public int hashCode() + { + return Objects.hash(getName(), getColumns(), isEnforced(), isRely()); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + TableConstraint constraint = (TableConstraint) obj; + return Objects.equals(this.getName(), constraint.getName()) && + Objects.equals(this.getColumns(), constraint.getColumns()) && + Objects.equals(this.isEnforced(), constraint.isEnforced()) && + Objects.equals(this.isRely(), constraint.isRely()); + } + + @Override + public String toString() + { + StringBuilder stringBuilder = new StringBuilder(this.getClass().getSimpleName()); + stringBuilder.append(" {"); + stringBuilder.append("name='").append(name.orElse("null")).append('\''); + stringBuilder.append(", columns='").append(columns).append('\''); + stringBuilder.append(", enforced='").append(enforced).append('\''); + stringBuilder.append(", rely='").append(rely).append('\''); + stringBuilder.append('}'); + return stringBuilder.toString(); + } +} diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/constraints/UniqueConstraint.java b/presto-spi/src/main/java/com/facebook/presto/spi/constraints/UniqueConstraint.java new file mode 100644 index 0000000000000..8d25a44ba8a6c --- /dev/null +++ b/presto-spi/src/main/java/com/facebook/presto/spi/constraints/UniqueConstraint.java @@ -0,0 +1,60 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.spi.constraints; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +public class UniqueConstraint + extends TableConstraint +{ + public UniqueConstraint(Set columnNames, boolean enabled, boolean rely) + { + this(Optional.empty(), columnNames, enabled, rely); + } + + @JsonCreator + public UniqueConstraint( + @JsonProperty("name") String name, + @JsonProperty("columns") Set columnNames, + @JsonProperty("enforced") boolean enabled, + @JsonProperty("rely") boolean rely) + { + super(Optional.of(name), columnNames, enabled, rely); + } + + protected UniqueConstraint(Optional name, Set columnNames, boolean enabled, boolean rely) + { + super(name, columnNames, enabled, rely); + } + + @Override + public Optional> rebaseConstraint(Map assignments) + { + if (this.getColumns().stream().allMatch(assignments::containsKey)) { + return Optional.of(new UniqueConstraint(getName(), + this.getColumns().stream().map(assignments::get).collect(Collectors.toSet()), + this.isEnforced(), + this.isRely())); + } + else { + return Optional.empty(); + } + } +} diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/plan/AggregationNode.java b/presto-spi/src/main/java/com/facebook/presto/spi/plan/AggregationNode.java index fbaedeb616d08..f770c3db8fa46 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/plan/AggregationNode.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/plan/AggregationNode.java @@ -96,6 +96,16 @@ public AggregationNode( this.outputs = unmodifiableList(outputs); } + /** + * Whether this node corresponds to a DISTINCT operation in SQL + */ + public static boolean isDistinct(AggregationNode node) + { + return node.getAggregations().isEmpty() && + node.getOutputVariables().size() == node.getGroupingKeys().size() && + node.getOutputVariables().containsAll(node.getGroupingKeys()); + } + public List getGroupingKeys() { return groupingSets.getGroupingKeys(); @@ -136,6 +146,13 @@ public List getSources() return unmodifiableList(Collections.singletonList(source)); } + @Override + public LogicalProperties computeLogicalProperties(LogicalPropertiesProvider logicalPropertiesProvider) + { + requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider cannot be null."); + return logicalPropertiesProvider.getAggregationProperties(this); + } + @Override public List getOutputVariables() { @@ -407,6 +424,17 @@ public Aggregation( this.mask = requireNonNull(mask, "mask is null"); } + public static AggregationNode.Aggregation removeDistinct(AggregationNode.Aggregation aggregation) + { + checkArgument(aggregation.isDistinct(), "Expected aggregation to have DISTINCT input"); + return new AggregationNode.Aggregation( + aggregation.getCall(), + aggregation.getFilter(), + aggregation.getOrderBy(), + false, + aggregation.getMask()); + } + @JsonProperty public CallExpression getCall() { diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/plan/DistinctLimitNode.java b/presto-spi/src/main/java/com/facebook/presto/spi/plan/DistinctLimitNode.java index bd5a3e1abf1aa..0f8b16635faa4 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/plan/DistinctLimitNode.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/plan/DistinctLimitNode.java @@ -94,6 +94,13 @@ public List getDistinctVariables() return distinctVariables; } + @Override + public LogicalProperties computeLogicalProperties(LogicalPropertiesProvider logicalPropertiesProvider) + { + requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider cannot be null."); + return logicalPropertiesProvider.getDistinctLimitProperties(this); + } + @Override public List getOutputVariables() { diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/plan/FilterNode.java b/presto-spi/src/main/java/com/facebook/presto/spi/plan/FilterNode.java index 4dcd4253d589a..0175a2f774f5b 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/plan/FilterNode.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/plan/FilterNode.java @@ -27,6 +27,7 @@ import static java.util.Collections.singletonList; import static java.util.Collections.unmodifiableList; +import static java.util.Objects.requireNonNull; @Immutable public final class FilterNode @@ -95,6 +96,13 @@ public PlanNode replaceChildren(List newChildren) return new FilterNode(getSourceLocation(), getId(), newChildren.get(0), predicate); } + @Override + public LogicalProperties computeLogicalProperties(LogicalPropertiesProvider logicalPropertiesProvider) + { + requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider cannot be null."); + return logicalPropertiesProvider.getFilterProperties(this); + } + @Override public boolean equals(Object o) { diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/plan/LimitNode.java b/presto-spi/src/main/java/com/facebook/presto/spi/plan/LimitNode.java index 0b6cd1f552d35..6c0aea79b1ec1 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/plan/LimitNode.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/plan/LimitNode.java @@ -101,6 +101,13 @@ public boolean isPartial() return step == Step.PARTIAL; } + @Override + public LogicalProperties computeLogicalProperties(LogicalPropertiesProvider logicalPropertiesProvider) + { + requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider cannot be null."); + return logicalPropertiesProvider.getLimitProperties(this); + } + @Override public List getOutputVariables() { diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/plan/LogicalProperties.java b/presto-spi/src/main/java/com/facebook/presto/spi/plan/LogicalProperties.java new file mode 100644 index 0000000000000..a7e060c7c6ede --- /dev/null +++ b/presto-spi/src/main/java/com/facebook/presto/spi/plan/LogicalProperties.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 com.facebook.presto.spi.plan; + +import com.facebook.presto.spi.relation.VariableReferenceExpression; + +import java.util.Set; + +/** + * Logical properties represent constraints that hold for a final or intermediate result produced by a PlanNode. + * They are used by the optimizer to enable query transformations that lead to better performing plans. + * For example, unique constraints can be used to eliminate redundant distinct operations. + * This is the interface used by optimizer rules to perform optimizations based upon logical properties. + *

+ * In this initial phase logical properties computed are based solely upon key constraints. In the future + * support for referential constraints, functional dependencies, and others will be added and + * hence this interface will become more robust over time. + */ +public interface LogicalProperties +{ + /** + * Determines if the set of variables form a unique constraint for a final or + * intermediate result produced by a PlanNode. + * + * @param keyVars + * @return True if the set of variables form a unique constraint or false otherwise. + */ + boolean isDistinct(Set keyVars); + + /** + * Determines if there is provably at most one tuple in a final or + * intermediate result set produced by a PlanNode. + * + * @return True if there is provably at most one tuple or false otherwise. + */ + boolean isAtMostSingleRow(); + + /** + * Determines if there is provably at most n tuples in a final or + * intermediate result set produced by a PlanNode. + * + * @return True if there is provably at most one tuple or false otherwise. + */ + boolean isAtMost(long n); +} diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/plan/LogicalPropertiesProvider.java b/presto-spi/src/main/java/com/facebook/presto/spi/plan/LogicalPropertiesProvider.java new file mode 100644 index 0000000000000..700c756363892 --- /dev/null +++ b/presto-spi/src/main/java/com/facebook/presto/spi/plan/LogicalPropertiesProvider.java @@ -0,0 +1,55 @@ +/* + * 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 com.facebook.presto.spi.plan; + +/** + * Defines a suite of plan node-specific methods for the computation of logical properties. + * Supplies a default implementation that produces an empty set of logical properties, and additionally, + * a suite of plan-node specific overrides of the default implementation. The implementations leverage + * property propagation builders supplied by LogicalPropertiesImpl. The LogicalPropertiesProvider + * mechanism enables a plan node to receive its logical property compute capabilities via dependency injection. + * This is needed because the computation of logical properties requires analysis of plan node's argument expressions, + * and the code the performs this analysis must be built in presto-main as this is where expression related classes are visible. + * The property computation implementation is dynamically injected into the presto-spi and presto-main plan node method's responsible + * for computing logical properties. This interface is defined in presto-spi where it is visible to all plan nodes. The + * implementation LogicalPropertiesImpl is provided in presto-main as per the reasons described above. + */ +public interface LogicalPropertiesProvider +{ + LogicalProperties getDefaultProperties(); + + LogicalProperties getTableScanProperties(TableScanNode tableScanNode); + + LogicalProperties getProjectProperties(ProjectNode projectNode); + + LogicalProperties getFilterProperties(FilterNode filterNode); + + LogicalProperties getJoinProperties(PlanNode joinNode); + + LogicalProperties getSemiJoinProperties(PlanNode semiJNode); + + LogicalProperties getSortProperties(PlanNode sortNode); + + LogicalProperties getAggregationProperties(AggregationNode aggregationNode); + + LogicalProperties getLimitProperties(LimitNode limitNode); + + LogicalProperties getTopNProperties(TopNNode limitNode); + + LogicalProperties getValuesProperties(ValuesNode valuesNode); + + LogicalProperties getDistinctLimitProperties(DistinctLimitNode limitNode); + + LogicalProperties getAssignUniqueIdProperties(PlanNode assignUniqueId); +} diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/plan/PlanNode.java b/presto-spi/src/main/java/com/facebook/presto/spi/plan/PlanNode.java index aca3db7fbc395..ee9d036ac604e 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/plan/PlanNode.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/plan/PlanNode.java @@ -57,6 +57,15 @@ public Optional getSourceLocation() */ public abstract List getSources(); + /** + * Logical properties are a function of source properties and the operation performed by the plan node + */ + public LogicalProperties computeLogicalProperties(LogicalPropertiesProvider logicalPropertiesProvider) + { + requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider cannot be null."); + return logicalPropertiesProvider.getDefaultProperties(); + } + /** * The output from the upstream PlanNodes. * It should serve as the input for the current PlanNode. diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/plan/ProjectNode.java b/presto-spi/src/main/java/com/facebook/presto/spi/plan/ProjectNode.java index a25530d5653d2..20a476f5dd026 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/plan/ProjectNode.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/plan/ProjectNode.java @@ -107,6 +107,13 @@ public PlanNode replaceChildren(List newChildren) return new ProjectNode(getSourceLocation(), getId(), newChildren.get(0), assignments, locality); } + @Override + public LogicalProperties computeLogicalProperties(LogicalPropertiesProvider logicalPropertiesProvider) + { + requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider cannot be null."); + return logicalPropertiesProvider.getProjectProperties(this); + } + @Override public boolean equals(Object o) { diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/plan/TableScanNode.java b/presto-spi/src/main/java/com/facebook/presto/spi/plan/TableScanNode.java index 290647a7a8326..24037ccaa3cd9 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/plan/TableScanNode.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/plan/TableScanNode.java @@ -17,6 +17,7 @@ import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.SourceLocation; import com.facebook.presto.spi.TableHandle; +import com.facebook.presto.spi.constraints.TableConstraint; import com.facebook.presto.spi.relation.VariableReferenceExpression; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -46,8 +47,8 @@ public final class TableScanNode // TODO: think about how to get rid of this in new planner // TODO: these two fields will not be effective if they are created by connectors until we have refactored PickTableLayout private final TupleDomain currentConstraint; - private final TupleDomain enforcedConstraint; + private final List> tableConstraints; /** * This constructor is for JSON deserialization only. Do not use! @@ -67,6 +68,7 @@ public TableScanNode( checkArgument(assignments.keySet().containsAll(outputVariables), "assignments does not cover all of outputs"); this.currentConstraint = null; this.enforcedConstraint = null; + this.tableConstraints = emptyList(); } public TableScanNode( @@ -77,6 +79,19 @@ public TableScanNode( Map assignments, TupleDomain currentConstraint, TupleDomain enforcedConstraint) + { + this (sourceLocation, id, table, outputVariables, assignments, emptyList(), currentConstraint, enforcedConstraint); + } + + public TableScanNode( + Optional sourceLocation, + PlanNodeId id, + TableHandle table, + List outputVariables, + Map assignments, + List> tableConstraints, + TupleDomain currentConstraint, + TupleDomain enforcedConstraint) { super(sourceLocation, id); this.table = requireNonNull(table, "table is null"); @@ -88,6 +103,7 @@ public TableScanNode( if (!currentConstraint.isAll() || !enforcedConstraint.isAll()) { checkArgument(table.getLayout().isPresent(), "tableLayout must be present when currentConstraint or enforcedConstraint is non-trivial"); } + this.tableConstraints = requireNonNull(tableConstraints, "tableConstraints is null"); } /** @@ -99,6 +115,14 @@ public TableHandle getTable() return table; } + /** + * Get table constraints defined by connector + */ + public List> getTableConstraints() + { + return tableConstraints; + } + /** * Get the mapping from symbols to columns */ @@ -145,6 +169,13 @@ public List getSources() return emptyList(); } + @Override + public LogicalProperties computeLogicalProperties(LogicalPropertiesProvider logicalPropertiesProvider) + { + requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider cannot be null."); + return logicalPropertiesProvider.getTableScanProperties(this); + } + @Override @JsonProperty public List getOutputVariables() diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/plan/TopNNode.java b/presto-spi/src/main/java/com/facebook/presto/spi/plan/TopNNode.java index c5b1f7baad689..267b600d7fca3 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/plan/TopNNode.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/plan/TopNNode.java @@ -88,6 +88,13 @@ public PlanNode getSource() return source; } + @Override + public LogicalProperties computeLogicalProperties(LogicalPropertiesProvider logicalPropertiesProvider) + { + requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider cannot be null."); + return logicalPropertiesProvider.getTopNProperties(this); + } + @Override public List getOutputVariables() { diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/plan/ValuesNode.java b/presto-spi/src/main/java/com/facebook/presto/spi/plan/ValuesNode.java index e22f45d609c92..1f5f25a226565 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/plan/ValuesNode.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/plan/ValuesNode.java @@ -62,6 +62,13 @@ public List> getRows() return rows; } + @Override + public LogicalProperties computeLogicalProperties(LogicalPropertiesProvider logicalPropertiesProvider) + { + requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider cannot be null."); + return logicalPropertiesProvider.getValuesProperties(this); + } + @Override @JsonProperty public List getOutputVariables() diff --git a/presto-tpch/src/main/java/com/facebook/presto/tpch/TpchConnectorFactory.java b/presto-tpch/src/main/java/com/facebook/presto/tpch/TpchConnectorFactory.java index 94cc256fefba0..56daec470e417 100644 --- a/presto-tpch/src/main/java/com/facebook/presto/tpch/TpchConnectorFactory.java +++ b/presto-tpch/src/main/java/com/facebook/presto/tpch/TpchConnectorFactory.java @@ -108,7 +108,7 @@ public ConnectorNodePartitioningProvider getNodePartitioningProvider() }; } - private int getSplitsPerNode(Map properties) + protected int getSplitsPerNode(Map properties) { try { return Integer.parseInt(firstNonNull(properties.get("tpch.splits-per-node"), String.valueOf(defaultSplitsPerNode))); @@ -118,8 +118,13 @@ private int getSplitsPerNode(Map properties) } } - private boolean isPartitioningEnabled(Map properties) + protected boolean isPartitioningEnabled(Map properties) { return Boolean.parseBoolean(properties.getOrDefault("tpch.partitioning-enabled", String.valueOf(partitioningEnabled))); } + + protected boolean isPredicatePushdownEnabled() + { + return predicatePushdownEnabled; + } } diff --git a/presto-tpch/src/main/java/com/facebook/presto/tpch/TpchMetadata.java b/presto-tpch/src/main/java/com/facebook/presto/tpch/TpchMetadata.java index 2fcfe2ec699a6..07ea444ab7546 100644 --- a/presto-tpch/src/main/java/com/facebook/presto/tpch/TpchMetadata.java +++ b/presto-tpch/src/main/java/com/facebook/presto/tpch/TpchMetadata.java @@ -338,6 +338,11 @@ public TableStatistics getTableStatistics(ConnectorSession session, ConnectorTab .orElse(TableStatistics.empty()); } + protected ColumnNaming getColumnNaming() + { + return columnNaming; + } + private Map, List> getColumnValuesRestrictions(TpchTable tpchTable, Constraint constraint) { TupleDomain constraintSummary = constraint.getSummary(); @@ -498,7 +503,7 @@ private List getSchemaNames(ConnectorSession session, Optional s return ImmutableList.of(); } - private static String scaleFactorSchemaName(double scaleFactor) + protected static String scaleFactorSchemaName(double scaleFactor) { return "sf" + scaleFactor; }