From 5b7278764c89c1823156c4bb71350fb0a9bede9e Mon Sep 17 00:00:00 2001 From: Eric Hwang Date: Fri, 22 Nov 2013 19:08:53 -0800 Subject: [PATCH] Add support for range predicate pushdowns This work is a complete refactor of how we generate Partitions in the planning/execution phases. Instead of generating Partitions after planning, we now integrate the generation directly into the optimization planning phase so that we can iterate our plan based on data returned by the Partitions that we discover. This not only provides a number of opportunities for optimization, but also significantly simplifies the amount of cross-talk between the split generation and the planning phases. As a result of these changes, it makes it much easier to build native range predicate push downs. --- .../presto/example/ExamplePartition.java | 9 +- .../presto/example/ExampleSplitManager.java | 10 +- .../facebook/presto/hive/HiveBucketing.java | 4 +- .../com/facebook/presto/hive/HiveClient.java | 74 +- .../facebook/presto/hive/HivePartition.java | 12 +- .../presto/hive/AbstractTestHiveClient.java | 88 +- .../connector/dual/DualSplitManager.java | 16 +- .../InformationSchemaDataStreamProvider.java | 8 +- .../InformationSchemaSplitManager.java | 27 +- .../presto/connector/jmx/JmxSplitManager.java | 16 +- .../connector/system/SystemSplitManager.java | 31 +- .../presto/execution/SqlQueryExecution.java | 4 +- .../presto/split/NativeSplitManager.java | 23 +- .../presto/split/PartitionFunction.java | 20 +- .../facebook/presto/split/SplitManager.java | 122 +- .../facebook/presto/sql/ExpressionUtils.java | 65 +- .../planner/DistributedExecutionPlanner.java | 32 +- .../presto/sql/planner/DomainTranslator.java | 557 +++++++++ .../presto/sql/planner/DomainUtils.java | 50 + .../planner/EffectivePredicateExtractor.java | 15 +- .../sql/planner/ExpressionInterpreter.java | 115 +- .../sql/planner/LiteralInterpreter.java | 174 +++ .../presto/sql/planner/LogicalPlanner.java | 4 +- .../sql/planner/PlanOptimizersFactory.java | 6 +- .../presto/sql/planner/PlanPrinter.java | 11 +- .../presto/sql/planner/PlanSanityChecker.java | 2 - .../presto/sql/planner/QueryPlanner.java | 4 +- .../presto/sql/planner/RelationPlanner.java | 5 +- .../optimizations/PredicatePushDown.java | 118 +- .../PruneUnreferencedOutputs.java | 17 +- .../optimizations/SimplifyExpressions.java | 10 +- .../optimizations/TableAliasSelector.java | 2 +- .../UnaliasSymbolReferences.java | 2 +- .../sql/planner/plan/TableScanNode.java | 134 ++- .../presto/tpch/TpchSplitManager.java | 16 +- .../facebook/presto/util/GraphvizPrinter.java | 3 +- .../facebook/presto/AbstractTestQueries.java | 7 +- .../execution/TestSqlStageExecution.java | 8 +- .../presto/execution/TestSqlTaskManager.java | 11 +- .../presto/split/TestNativeSplitManager.java | 88 +- .../sql/planner/TestDomainTranslator.java | 1020 +++++++++++++++++ .../TestEffectivePredicateExtractor.java | 188 ++- .../sql/planner/TestingColumnHandle.java | 62 + .../presto/util/LocalQueryRunner.java | 27 +- presto-spi/pom.xml | 22 + .../com/facebook/presto/spi/ColumnType.java | 29 +- .../presto/spi/ConnectorSplitManager.java | 19 +- .../java/com/facebook/presto/spi/Domain.java | 207 ++++ .../java/com/facebook/presto/spi/Marker.java | 296 +++++ .../com/facebook/presto/spi/Partition.java | 8 +- .../facebook/presto/spi/PartitionResult.java | 48 + .../java/com/facebook/presto/spi/Range.java | 225 ++++ .../presto/spi/SerializableNativeValue.java | 157 +++ .../facebook/presto/spi/SortedRangeSet.java | 359 ++++++ .../com/facebook/presto/spi/TupleDomain.java | 335 ++++++ .../ClassLoaderSafeConnectorSplitManager.java | 8 +- .../com/facebook/presto/spi/TestDomain.java | 380 ++++++ .../com/facebook/presto/spi/TestMarker.java | 176 +++ .../com/facebook/presto/spi/TestRange.java | 282 +++++ .../presto/spi/TestSortedRangeSet.java | 457 ++++++++ .../facebook/presto/spi/TestTupleDomain.java | 613 ++++++++++ .../presto/spi/TestingColumnHandle.java | 64 ++ 62 files changed, 6254 insertions(+), 648 deletions(-) create mode 100644 presto-main/src/main/java/com/facebook/presto/sql/planner/DomainTranslator.java create mode 100644 presto-main/src/main/java/com/facebook/presto/sql/planner/DomainUtils.java create mode 100644 presto-main/src/main/java/com/facebook/presto/sql/planner/LiteralInterpreter.java create mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/TestDomainTranslator.java create mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/TestingColumnHandle.java create mode 100644 presto-spi/src/main/java/com/facebook/presto/spi/Domain.java create mode 100644 presto-spi/src/main/java/com/facebook/presto/spi/Marker.java create mode 100644 presto-spi/src/main/java/com/facebook/presto/spi/PartitionResult.java create mode 100644 presto-spi/src/main/java/com/facebook/presto/spi/Range.java create mode 100644 presto-spi/src/main/java/com/facebook/presto/spi/SerializableNativeValue.java create mode 100644 presto-spi/src/main/java/com/facebook/presto/spi/SortedRangeSet.java create mode 100644 presto-spi/src/main/java/com/facebook/presto/spi/TupleDomain.java create mode 100644 presto-spi/src/test/java/com/facebook/presto/spi/TestDomain.java create mode 100644 presto-spi/src/test/java/com/facebook/presto/spi/TestMarker.java create mode 100644 presto-spi/src/test/java/com/facebook/presto/spi/TestRange.java create mode 100644 presto-spi/src/test/java/com/facebook/presto/spi/TestSortedRangeSet.java create mode 100644 presto-spi/src/test/java/com/facebook/presto/spi/TestTupleDomain.java create mode 100644 presto-spi/src/test/java/com/facebook/presto/spi/TestingColumnHandle.java diff --git a/presto-example-http/src/main/java/com/facebook/presto/example/ExamplePartition.java b/presto-example-http/src/main/java/com/facebook/presto/example/ExamplePartition.java index 5b5e985d19324..87a42bed619c6 100644 --- a/presto-example-http/src/main/java/com/facebook/presto/example/ExamplePartition.java +++ b/presto-example-http/src/main/java/com/facebook/presto/example/ExamplePartition.java @@ -13,12 +13,9 @@ */ package com.facebook.presto.example; -import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.Partition; +import com.facebook.presto.spi.TupleDomain; import com.google.common.base.Objects; -import com.google.common.collect.ImmutableMap; - -import java.util.Map; import static com.google.common.base.Preconditions.checkNotNull; @@ -51,9 +48,9 @@ public String getTableName() } @Override - public Map getKeys() + public TupleDomain getTupleDomain() { - return ImmutableMap.of(); + return TupleDomain.all(); } @Override diff --git a/presto-example-http/src/main/java/com/facebook/presto/example/ExampleSplitManager.java b/presto-example-http/src/main/java/com/facebook/presto/example/ExampleSplitManager.java index a21d3eb9a02c9..b9b22ad56f427 100644 --- a/presto-example-http/src/main/java/com/facebook/presto/example/ExampleSplitManager.java +++ b/presto-example-http/src/main/java/com/facebook/presto/example/ExampleSplitManager.java @@ -13,11 +13,12 @@ */ package com.facebook.presto.example; -import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.ConnectorSplitManager; import com.facebook.presto.spi.Partition; +import com.facebook.presto.spi.PartitionResult; import com.facebook.presto.spi.Split; import com.facebook.presto.spi.TableHandle; +import com.facebook.presto.spi.TupleDomain; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; @@ -26,7 +27,6 @@ import java.net.URI; import java.util.Collections; import java.util.List; -import java.util.Map; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; @@ -58,13 +58,15 @@ public boolean canHandle(TableHandle tableHandle) } @Override - public List getPartitions(TableHandle tableHandle, Map bindings) + public PartitionResult getPartitions(TableHandle tableHandle, TupleDomain tupleDomain) { checkArgument(tableHandle instanceof ExampleTableHandle, "tableHandle is not an instance of ExampleTableHandle"); ExampleTableHandle exampleTableHandle = (ExampleTableHandle) tableHandle; // example connector has only one partition - return ImmutableList.of(new ExamplePartition(exampleTableHandle.getSchemaName(), exampleTableHandle.getTableName())); + List partitions = ImmutableList.of(new ExamplePartition(exampleTableHandle.getSchemaName(), exampleTableHandle.getTableName())); + // example connector does not do any additional processing/filtering with the TupleDomain, so just return the whole TupleDomain + return new PartitionResult(partitions, tupleDomain); } @Override diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveBucketing.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveBucketing.java index d74ec8a673b0f..c8e34e1b93643 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveBucketing.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveBucketing.java @@ -65,7 +65,7 @@ final class HiveBucketing private HiveBucketing() {} - public static Optional getBucketNumber(Table table, Map bindings) + public static Optional getBucketNumber(Table table, Map bindings) { if (!table.getSd().isSetBucketCols() || table.getSd().getBucketCols().isEmpty() || !table.getSd().isSetNumBuckets() || table.getSd().getNumBuckets() <= 0 || @@ -99,7 +99,7 @@ public static Optional getBucketNumber(Table table, Map bucketBindings = new HashMap<>(); - for (Entry entry : bindings.entrySet()) { + for (Entry entry : bindings.entrySet()) { HiveColumnHandle colHandle = (HiveColumnHandle) entry.getKey(); if (bucketColumns.contains(colHandle.getName())) { bucketBindings.put(colHandle.getName(), entry.getValue()); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveClient.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveClient.java index 5278d0531ca7f..5ba9a05dafec3 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveClient.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveClient.java @@ -21,23 +21,29 @@ import com.facebook.presto.spi.ConnectorRecordSetProvider; import com.facebook.presto.spi.ConnectorSplitManager; import com.facebook.presto.spi.ConnectorTableMetadata; +import com.facebook.presto.spi.Domain; import com.facebook.presto.spi.Partition; +import com.facebook.presto.spi.PartitionResult; +import com.facebook.presto.spi.Range; import com.facebook.presto.spi.RecordSet; import com.facebook.presto.spi.SchemaTableName; import com.facebook.presto.spi.SchemaTablePrefix; import com.facebook.presto.spi.Split; import com.facebook.presto.spi.TableHandle; import com.facebook.presto.spi.TableNotFoundException; +import com.facebook.presto.spi.TupleDomain; import com.google.common.base.Function; import com.google.common.base.Objects; import com.google.common.base.Optional; import com.google.common.base.Predicate; import com.google.common.base.Throwables; import com.google.common.collect.AbstractIterator; +import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.inject.Inject; import io.airlift.log.Logger; @@ -73,6 +79,8 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; +import static com.google.common.base.Predicates.in; +import static com.google.common.base.Predicates.not; import static com.google.common.collect.Iterables.concat; import static com.google.common.collect.Iterables.transform; import static java.lang.Boolean.parseBoolean; @@ -329,10 +337,10 @@ public void dropTable(TableHandle tableHandle) } @Override - public List getPartitions(TableHandle tableHandle, Map bindings) + public PartitionResult getPartitions(TableHandle tableHandle, TupleDomain tupleDomain) { checkNotNull(tableHandle, "tableHandle is null"); - checkNotNull(bindings, "bindings is null"); + checkNotNull(tupleDomain, "tupleDomain is null"); SchemaTableName tableName = getTableName(tableHandle); List partitionKeys; @@ -347,27 +355,32 @@ public List getPartitions(TableHandle tableHandle, Map partitionKeysByName = ImmutableMap.builder(); + ImmutableMap.Builder partitionKeysByNameBuilder = ImmutableMap.builder(); List filterPrefix = new ArrayList<>(); for (int i = 0; i < partitionKeys.size(); i++) { FieldSchema field = partitionKeys.get(i); HiveColumnHandle columnHandle = new HiveColumnHandle(connectorId, field.getName(), i, getSupportedHiveType(field.getType()), -1, true); - partitionKeysByName.put(field.getName(), columnHandle); + partitionKeysByNameBuilder.put(field.getName(), columnHandle); // only add to prefix if all previous keys have a value - if (filterPrefix.size() == i) { - Object value = bindings.get(columnHandle); - if (value != null) { - checkArgument(value instanceof Boolean || value instanceof String || value instanceof Double || value instanceof Long, - "Only Boolean, String, Double and Long partition keys are supported"); - filterPrefix.add(value.toString()); + if (filterPrefix.size() == i && !tupleDomain.isNone()) { + Domain domain = tupleDomain.getDomains().get(columnHandle); + if (domain != null && domain.getRanges().getRangeCount() == 1) { + // We intentionally ignore whether NULL is in the domain since partition keys can never be NULL + Range range = Iterables.getOnlyElement(domain.getRanges()); + if (range.isSingleValue()) { + Comparable value = range.getLow().getValue(); + checkArgument(value instanceof Boolean || value instanceof String || value instanceof Double || value instanceof Long, + "Only Boolean, String, Double and Long partition keys are supported"); + filterPrefix.add(value.toString()); + } } } } @@ -390,8 +403,20 @@ else if (filterPrefix.isEmpty()) { } // do a final pass to filter based on fields that could not be used to build the prefix - Iterable partitions = transform(partitionNames, toPartition(tableName, partitionKeysByName.build(), bucket)); - return ImmutableList.copyOf(Iterables.filter(partitions, partitionMatches(bindings))); + Map partitionKeysByName = partitionKeysByNameBuilder.build(); + List partitions = FluentIterable.from(partitionNames) + .transform(toPartition(tableName, partitionKeysByName, bucket)) + .filter(partitionMatches(tupleDomain)) + .filter(Partition.class) + .toList(); + + // All partition key domains will be fully evaluated, so we don't need to include those + TupleDomain remainingTupleDomain = TupleDomain.none(); + if (!tupleDomain.isNone()) { + remainingTupleDomain = TupleDomain.withColumnDomains(Maps.filterKeys(tupleDomain.getDomains(), not(in(partitionKeysByName.values())))); + } + + return new PartitionResult(partitions, remainingTupleDomain); } @Override @@ -539,15 +564,15 @@ public String toString() .toString(); } - private static Function toPartition( + private static Function toPartition( final SchemaTableName tableName, final Map columnsByName, final Optional bucket) { - return new Function() + return new Function() { @Override - public Partition apply(String partitionId) + public HivePartition apply(String partitionId) { try { if (partitionId.equals(UNPARTITIONED_ID)) { @@ -555,7 +580,7 @@ public Partition apply(String partitionId) } LinkedHashMap keys = Warehouse.makeSpecFromName(partitionId); - ImmutableMap.Builder builder = ImmutableMap.builder(); + ImmutableMap.Builder> builder = ImmutableMap.builder(); for (Entry entry : keys.entrySet()) { ColumnHandle columnHandle = columnsByName.get(entry.getKey()); checkArgument(columnHandle != null, "Invalid partition key %s in partition %s", entry.getKey(), partitionId); @@ -607,16 +632,19 @@ else if (hiveColumnHandle.getHiveType() == HiveType.TIMESTAMP) { }; } - public static Predicate partitionMatches(final Map filters) + public static Predicate partitionMatches(final TupleDomain tupleDomain) { - return new Predicate() + return new Predicate() { @Override - public boolean apply(Partition partition) + public boolean apply(HivePartition partition) { - for (Map.Entry entry : partition.getKeys().entrySet()) { - Object filterValue = filters.get(entry.getKey()); - if (filterValue != null && !entry.getValue().equals(filterValue)) { + if (tupleDomain.isNone()) { + return false; + } + for (Entry> entry : partition.getKeys().entrySet()) { + Domain allowedDomain = tupleDomain.getDomains().get(entry.getKey()); + if (allowedDomain != null && !allowedDomain.includesValue(entry.getValue())) { return false; } } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HivePartition.java b/presto-hive/src/main/java/com/facebook/presto/hive/HivePartition.java index 1e3ca80f22da6..192d6bc802250 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HivePartition.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HivePartition.java @@ -16,6 +16,7 @@ import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.Partition; import com.facebook.presto.spi.SchemaTableName; +import com.facebook.presto.spi.TupleDomain; import com.google.common.base.Optional; import com.google.common.collect.ImmutableMap; @@ -31,7 +32,7 @@ public class HivePartition private final SchemaTableName tableName; private final String partitionId; - private final Map keys; + private final Map> keys; private final Optional bucket; public HivePartition(SchemaTableName tableName) @@ -42,7 +43,7 @@ public HivePartition(SchemaTableName tableName) this.bucket = Optional.absent(); } - public HivePartition(SchemaTableName tableName, String partitionId, Map keys, Optional bucket) + public HivePartition(SchemaTableName tableName, String partitionId, Map> keys, Optional bucket) { this.tableName = checkNotNull(tableName, "tableName is null"); this.partitionId = checkNotNull(partitionId, "partitionId is null"); @@ -62,7 +63,12 @@ public String getPartitionId() } @Override - public Map getKeys() + public TupleDomain getTupleDomain() + { + return TupleDomain.withFixedValues(keys); + } + + public Map> getKeys() { return keys; } 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 963a57918bb94..80a40b64a8615 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 @@ -20,7 +20,9 @@ import com.facebook.presto.spi.ConnectorRecordSetProvider; import com.facebook.presto.spi.ConnectorSplitManager; import com.facebook.presto.spi.ConnectorTableMetadata; +import com.facebook.presto.spi.Domain; import com.facebook.presto.spi.Partition; +import com.facebook.presto.spi.PartitionResult; import com.facebook.presto.spi.RecordCursor; import com.facebook.presto.spi.RecordSet; import com.facebook.presto.spi.SchemaNotFoundException; @@ -29,6 +31,7 @@ import com.facebook.presto.spi.Split; import com.facebook.presto.spi.TableHandle; import com.facebook.presto.spi.TableNotFoundException; +import com.facebook.presto.spi.TupleDomain; import com.google.common.base.Charsets; import com.google.common.base.Function; import com.google.common.base.Optional; @@ -42,7 +45,6 @@ import org.joda.time.DateTime; import org.testng.annotations.Test; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; @@ -119,22 +121,22 @@ protected void setupHive(String connectorId, String databaseName) partitions = ImmutableSet.of( new HivePartition(table, "ds=2012-12-29/file_format=rcfile-text/dummy=0", - ImmutableMap.of(dsColumn, "2012-12-29", fileFormatColumn, "rcfile-text", dummyColumn, 0L), + ImmutableMap.>of(dsColumn, "2012-12-29", fileFormatColumn, "rcfile-text", dummyColumn, 0L), Optional.absent()), new HivePartition(table, "ds=2012-12-29/file_format=rcfile-binary/dummy=2", - ImmutableMap.of(dsColumn, "2012-12-29", fileFormatColumn, "rcfile-binary", dummyColumn, 2L), + ImmutableMap.>of(dsColumn, "2012-12-29", fileFormatColumn, "rcfile-binary", dummyColumn, 2L), Optional.absent()), new HivePartition(table, "ds=2012-12-29/file_format=sequencefile/dummy=4", - ImmutableMap.of(dsColumn, "2012-12-29", fileFormatColumn, "sequencefile", dummyColumn, 4L), + ImmutableMap.>of(dsColumn, "2012-12-29", fileFormatColumn, "sequencefile", dummyColumn, 4L), Optional.absent()), new HivePartition(table, "ds=2012-12-29/file_format=textfile/dummy=6", - ImmutableMap.of(dsColumn, "2012-12-29", fileFormatColumn, "textfile", dummyColumn, 6L), + ImmutableMap.>of(dsColumn, "2012-12-29", fileFormatColumn, "textfile", dummyColumn, 6L), Optional.absent())); unpartitionedPartitions = ImmutableSet.of(new HivePartition(tableUnpartitioned)); - invalidPartition = new HivePartition(invalidTable, "unknown", ImmutableMap.of(), Optional. absent()); + invalidPartition = new HivePartition(invalidTable, "unknown", ImmutableMap.>of(), Optional. absent()); } protected void setup(String host, int port, String databaseName) @@ -210,8 +212,8 @@ public void testGetPartitions() throws Exception { TableHandle tableHandle = getTableHandle(table); - List partitions = splitManager.getPartitions(tableHandle, ImmutableMap.of()); - assertExpectedPartitions(partitions); + PartitionResult partitionResult = splitManager.getPartitions(tableHandle, TupleDomain.all()); + assertExpectedPartitions(partitionResult.getPartitions()); } @Test @@ -219,15 +221,15 @@ public void testGetPartitionsWithBindings() throws Exception { TableHandle tableHandle = getTableHandle(table); - List partitions = splitManager.getPartitions(tableHandle, ImmutableMap.of(intColumn, 5)); - assertExpectedPartitions(partitions); + PartitionResult partitionResult = splitManager.getPartitions(tableHandle, TupleDomain.withColumnDomains(ImmutableMap.of(intColumn, Domain.singleValue(5L)))); + assertExpectedPartitions(partitionResult.getPartitions()); } @Test(expectedExceptions = TableNotFoundException.class) public void testGetPartitionsException() throws Exception { - splitManager.getPartitions(invalidTableHandle, Collections.emptyMap()); + splitManager.getPartitions(invalidTableHandle, TupleDomain.all()); } @Test @@ -235,8 +237,8 @@ public void testGetPartitionNames() throws Exception { TableHandle tableHandle = getTableHandle(table); - List partitions = splitManager.getPartitions(tableHandle, ImmutableMap.of()); - assertExpectedPartitions(partitions); + PartitionResult partitionResult = splitManager.getPartitions(tableHandle, TupleDomain.all()); + assertExpectedPartitions(partitionResult.getPartitions()); } protected void assertExpectedPartitions(List actualPartitions) @@ -256,6 +258,7 @@ protected void assertExpectedPartitions(List actualPartitions) assertEquals(actualPartition.getKeys(), expectedPartition.getKeys()); assertEquals(actualPartition.getTableName(), expectedPartition.getTableName()); assertEquals(actualPartition.getBucket(), expectedPartition.getBucket()); + assertEquals(actualPartition.getTupleDomain(), expectedPartition.getTupleDomain()); } } @@ -264,16 +267,16 @@ public void testGetPartitionNamesUnpartitioned() throws Exception { TableHandle tableHandle = getTableHandle(tableUnpartitioned); - List partitions = splitManager.getPartitions(tableHandle, ImmutableMap.of()); - assertEquals(partitions.size(), 1); - assertEquals(partitions, unpartitionedPartitions); + PartitionResult partitionResult = splitManager.getPartitions(tableHandle, TupleDomain.all()); + assertEquals(partitionResult.getPartitions().size(), 1); + assertEquals(partitionResult.getPartitions(), unpartitionedPartitions); } @Test(expectedExceptions = TableNotFoundException.class) public void testGetPartitionNamesException() throws Exception { - splitManager.getPartitions(invalidTableHandle, ImmutableMap.of()); + splitManager.getPartitions(invalidTableHandle, TupleDomain.all()); } @SuppressWarnings({"ValueOfIncrementOrDecrementUsed", "UnusedAssignment"}) @@ -349,8 +352,8 @@ public void testGetPartitionSplitsBatch() throws Exception { TableHandle tableHandle = getTableHandle(table); - List partitions = splitManager.getPartitions(tableHandle, ImmutableMap.of()); - Iterable iterator = splitManager.getPartitionSplits(tableHandle, partitions); + PartitionResult partitionResult = splitManager.getPartitions(tableHandle, TupleDomain.all()); + Iterable iterator = splitManager.getPartitionSplits(tableHandle, partitionResult.getPartitions()); List splits = ImmutableList.copyOf(iterator); assertEquals(splits.size(), partitions.size()); @@ -361,8 +364,8 @@ public void testGetPartitionSplitsBatchUnpartitioned() throws Exception { TableHandle tableHandle = getTableHandle(tableUnpartitioned); - List partitions = splitManager.getPartitions(tableHandle, ImmutableMap.of()); - Iterable iterator = splitManager.getPartitionSplits(tableHandle, partitions); + PartitionResult partitionResult = splitManager.getPartitions(tableHandle, TupleDomain.all()); + Iterable iterator = splitManager.getPartitionSplits(tableHandle, partitionResult.getPartitions()); List splits = ImmutableList.copyOf(iterator); assertEquals(splits.size(), 1); @@ -390,7 +393,7 @@ public void testGetPartitionTableOffline() { TableHandle tableHandle = getTableHandle(tableOffline); try { - splitManager.getPartitions(tableHandle, ImmutableMap.of()); + splitManager.getPartitions(tableHandle, TupleDomain.all()); fail("expected TableOfflineException"); } catch (TableOfflineException e) { @@ -408,9 +411,10 @@ public void testGetPartitionSplitsTableOfflinePartition() ColumnHandle dsColumn = metadata.getColumnHandle(tableHandle, "ds"); assertNotNull(dsColumn); - List partitions = splitManager.getPartitions(tableHandle, ImmutableMap.of(dsColumn, "2012-12-30")); - for (Partition partition : partitions) { - if ("2012-12-30".equals(partition.getKeys().get(dsColumn))) { + TupleDomain tupleDomain = TupleDomain.withColumnDomains(ImmutableMap.of(dsColumn, Domain.singleValue("2012-12-30"))); + PartitionResult partitionResult = splitManager.getPartitions(tableHandle, tupleDomain); + for (Partition partition : partitionResult.getPartitions()) { + if (Domain.singleValue("2012-12-30").equals(partition.getTupleDomain().getDomains().get(dsColumn))) { try { Iterables.size(splitManager.getPartitionSplits(tableHandle, ImmutableList.of(partition))); fail("Expected PartitionOfflineException"); @@ -439,14 +443,14 @@ public void testBucketedTableStringInt() Long testSmallint = 412L; // Reverse the order of bindings as compared to bucketing order - ImmutableMap bindings = ImmutableMap.builder() + ImmutableMap> bindings = ImmutableMap.>builder() .put(columnHandles.get(columnIndex.get("t_int")), testInt) .put(columnHandles.get(columnIndex.get("t_string")), testString) .put(columnHandles.get(columnIndex.get("t_smallint")), testSmallint) .build(); - List partitions = splitManager.getPartitions(tableHandle, bindings); - List splits = ImmutableList.copyOf(splitManager.getPartitionSplits(tableHandle, partitions)); + PartitionResult partitionResult = splitManager.getPartitions(tableHandle, TupleDomain.withFixedValues(bindings)); + List splits = ImmutableList.copyOf(splitManager.getPartitionSplits(tableHandle, partitionResult.getPartitions())); assertEquals(splits.size(), 1); boolean rowFound = false; @@ -474,14 +478,14 @@ public void testBucketedTableBigintBoolean() Long testBigint = 605L; Boolean testBoolean = true; - ImmutableMap bindings = ImmutableMap.builder() + ImmutableMap> bindings = ImmutableMap.>builder() .put(columnHandles.get(columnIndex.get("t_string")), testString) .put(columnHandles.get(columnIndex.get("t_bigint")), testBigint) .put(columnHandles.get(columnIndex.get("t_boolean")), testBoolean) .build(); - List partitions = splitManager.getPartitions(tableHandle, bindings); - List splits = ImmutableList.copyOf(splitManager.getPartitionSplits(tableHandle, partitions)); + PartitionResult partitionResult = splitManager.getPartitions(tableHandle, TupleDomain.withFixedValues(bindings)); + List splits = ImmutableList.copyOf(splitManager.getPartitionSplits(tableHandle, partitionResult.getPartitions())); assertEquals(splits.size(), 1); boolean rowFound = false; @@ -506,13 +510,13 @@ public void testBucketedTableDoubleFloat() List columnHandles = ImmutableList.copyOf(metadata.getColumnHandles(tableHandle).values()); Map columnIndex = indexColumns(columnHandles); - ImmutableMap bindings = ImmutableMap.builder() + ImmutableMap> bindings = ImmutableMap.>builder() .put(columnHandles.get(columnIndex.get("t_float")), 406.1000061035156) .put(columnHandles.get(columnIndex.get("t_double")), 407.2) .build(); - List partitions = splitManager.getPartitions(tableHandle, bindings); - List splits = ImmutableList.copyOf(splitManager.getPartitionSplits(tableHandle, partitions)); + PartitionResult partitionResult = splitManager.getPartitions(tableHandle, TupleDomain.withFixedValues(bindings)); + List splits = ImmutableList.copyOf(splitManager.getPartitionSplits(tableHandle, partitionResult.getPartitions())); assertEquals(splits.size(), 32); int count = 0; @@ -535,8 +539,8 @@ public void testGetRecords() List columnHandles = ImmutableList.copyOf(metadata.getColumnHandles(tableHandle).values()); Map columnIndex = indexColumns(columnHandles); - List partitions = splitManager.getPartitions(tableHandle, ImmutableMap.of()); - List splits = ImmutableList.copyOf(splitManager.getPartitionSplits(tableHandle, partitions)); + PartitionResult partitionResult = splitManager.getPartitions(tableHandle, TupleDomain.all()); + List splits = ImmutableList.copyOf(splitManager.getPartitionSplits(tableHandle, partitionResult.getPartitions())); assertEquals(splits.size(), this.partitions.size()); for (Split split : splits) { HiveSplit hiveSplit = (HiveSplit) split; @@ -655,8 +659,8 @@ public void testGetPartialRecords() List columnHandles = ImmutableList.copyOf(metadata.getColumnHandles(tableHandle).values()); Map columnIndex = indexColumns(columnHandles); - List partitions = splitManager.getPartitions(tableHandle, ImmutableMap.of()); - List splits = ImmutableList.copyOf(splitManager.getPartitionSplits(tableHandle, partitions)); + PartitionResult partitionResult = splitManager.getPartitions(tableHandle, TupleDomain.all()); + List splits = ImmutableList.copyOf(splitManager.getPartitionSplits(tableHandle, partitionResult.getPartitions())); assertEquals(splits.size(), this.partitions.size()); for (Split split : splits) { HiveSplit hiveSplit = (HiveSplit) split; @@ -692,8 +696,8 @@ public void testGetRecordsUnpartitioned() List columnHandles = ImmutableList.copyOf(metadata.getColumnHandles(tableHandle).values()); Map columnIndex = indexColumns(columnHandles); - List partitions = splitManager.getPartitions(tableHandle, ImmutableMap.of()); - List splits = ImmutableList.copyOf(splitManager.getPartitionSplits(tableHandle, partitions)); + PartitionResult partitionResult = splitManager.getPartitions(tableHandle, TupleDomain.all()); + List splits = ImmutableList.copyOf(splitManager.getPartitionSplits(tableHandle, partitionResult.getPartitions())); assertEquals(splits.size(), 1); for (Split split : splits) { @@ -728,8 +732,8 @@ public void testGetRecordsInvalidColumn() throws Exception { TableHandle table = getTableHandle(tableUnpartitioned); - List partitions = splitManager.getPartitions(table, ImmutableMap.of()); - Split split = Iterables.getFirst(splitManager.getPartitionSplits(table, partitions), null); + PartitionResult partitionResult = splitManager.getPartitions(table, TupleDomain.all()); + Split split = Iterables.getFirst(splitManager.getPartitionSplits(table, partitionResult.getPartitions()), null); RecordSet recordSet = recordSetProvider.getRecordSet(split, ImmutableList.of(invalidColumnHandle)); recordSet.cursor(); } diff --git a/presto-main/src/main/java/com/facebook/presto/connector/dual/DualSplitManager.java b/presto-main/src/main/java/com/facebook/presto/connector/dual/DualSplitManager.java index c53dcd721c558..bc0bd6bbd7a0b 100644 --- a/presto-main/src/main/java/com/facebook/presto/connector/dual/DualSplitManager.java +++ b/presto-main/src/main/java/com/facebook/presto/connector/dual/DualSplitManager.java @@ -14,20 +14,19 @@ package com.facebook.presto.connector.dual; import com.facebook.presto.metadata.NodeManager; -import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.ConnectorSplitManager; import com.facebook.presto.spi.Partition; +import com.facebook.presto.spi.PartitionResult; import com.facebook.presto.spi.Split; import com.facebook.presto.spi.TableHandle; +import com.facebook.presto.spi.TupleDomain; import com.google.common.base.Objects; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import javax.inject.Inject; import java.util.List; -import java.util.Map; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; @@ -57,14 +56,15 @@ public boolean canHandle(TableHandle handle) } @Override - public List getPartitions(TableHandle table, Map bindings) + public PartitionResult getPartitions(TableHandle table, TupleDomain tupleDomain) { checkNotNull(table, "table is null"); - checkNotNull(bindings, "bindings is null"); + checkNotNull(tupleDomain, "tupleDomain is null"); checkArgument(table instanceof DualTableHandle, "TableHandle must be a DualTableHandle"); - return ImmutableList.of(new DualPartition()); + ImmutableList partitions = ImmutableList.of(new DualPartition()); + return new PartitionResult(partitions, tupleDomain); } @Override @@ -93,9 +93,9 @@ public String getPartitionId() } @Override - public Map getKeys() + public TupleDomain getTupleDomain() { - return ImmutableMap.of(); + return TupleDomain.all(); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/connector/informationSchema/InformationSchemaDataStreamProvider.java b/presto-main/src/main/java/com/facebook/presto/connector/informationSchema/InformationSchemaDataStreamProvider.java index dabf0dd0bc122..fe66dedc7aee3 100644 --- a/presto-main/src/main/java/com/facebook/presto/connector/informationSchema/InformationSchemaDataStreamProvider.java +++ b/presto-main/src/main/java/com/facebook/presto/connector/informationSchema/InformationSchemaDataStreamProvider.java @@ -25,9 +25,11 @@ import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.ColumnMetadata; import com.facebook.presto.spi.Partition; +import com.facebook.presto.spi.PartitionResult; import com.facebook.presto.spi.SchemaTableName; import com.facebook.presto.spi.Split; import com.facebook.presto.spi.TableHandle; +import com.facebook.presto.spi.TupleDomain; import com.facebook.presto.split.ConnectorDataStreamProvider; import com.facebook.presto.split.SplitManager; import com.facebook.presto.sql.analyzer.Type; @@ -219,10 +221,10 @@ private InternalTable buildPartitions(String catalogName, Map fi Optional tableHandle = metadata.getTableHandle(tableName); checkArgument(tableHandle.isPresent(), "Table %s does not exist", tableName); Map columnHandles = ImmutableBiMap.copyOf(metadata.getColumnHandles(tableHandle.get())).inverse(); - List partitions = splitManager.getPartitions(tableHandle.get(), Optional.>absent()); + PartitionResult partitionResult = splitManager.getPartitions(tableHandle.get(), Optional.absent()); - for (Partition partition : partitions) { - for (Map.Entry entry : partition.getKeys().entrySet()) { + for (Partition partition : partitionResult.getPartitions()) { + for (Entry> entry : partition.getTupleDomain().extractFixedValues().entrySet()) { ColumnHandle columnHandle = entry.getKey(); String columnName = columnHandles.get(columnHandle); String value = entry.getValue() != null ? String.valueOf(entry.getValue()) : null; diff --git a/presto-main/src/main/java/com/facebook/presto/connector/informationSchema/InformationSchemaSplitManager.java b/presto-main/src/main/java/com/facebook/presto/connector/informationSchema/InformationSchemaSplitManager.java index 0f9db7f75e5ad..1e0253018bbb9 100644 --- a/presto-main/src/main/java/com/facebook/presto/connector/informationSchema/InformationSchemaSplitManager.java +++ b/presto-main/src/main/java/com/facebook/presto/connector/informationSchema/InformationSchemaSplitManager.java @@ -18,8 +18,10 @@ import com.facebook.presto.spi.ConnectorSplitManager; import com.facebook.presto.spi.HostAddress; import com.facebook.presto.spi.Partition; +import com.facebook.presto.spi.PartitionResult; import com.facebook.presto.spi.Split; import com.facebook.presto.spi.TableHandle; +import com.facebook.presto.spi.TupleDomain; import com.google.common.base.Objects; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -59,15 +61,20 @@ public boolean canHandle(TableHandle handle) } @Override - public List getPartitions(TableHandle table, Map bindings) + public PartitionResult getPartitions(TableHandle table, TupleDomain tupleDomain) { checkNotNull(table, "table is null"); - checkNotNull(bindings, "bindings is null"); + checkNotNull(tupleDomain, "tupleDomain is null"); checkArgument(table instanceof InformationSchemaTableHandle, "TableHandle must be an InformationSchemaTableHandle"); InformationSchemaTableHandle informationSchemaTableHandle = (InformationSchemaTableHandle) table; - return ImmutableList.of(new InformationSchemaPartition(informationSchemaTableHandle, bindings)); + Map> bindings = tupleDomain.extractFixedValues(); + + List partitions = ImmutableList.of(new InformationSchemaPartition(informationSchemaTableHandle, bindings)); + // We don't strip out the bindings that we have created from the undeterminedTupleDomain b/c the current InformationSchema + // system requires that all filters be re-applied at execution time. + return new PartitionResult(partitions, tupleDomain); } @Override @@ -85,7 +92,7 @@ public Iterable getPartitionSplits(TableHandle table, List par List localAddress = ImmutableList.of(nodeManager.getCurrentNode().getHostAndPort()); ImmutableMap.Builder filters = ImmutableMap.builder(); - for (Entry entry : informationSchemaPartition.getFilters().entrySet()) { + for (Entry> entry : informationSchemaPartition.getFilters().entrySet()) { InformationSchemaColumnHandle informationSchemaColumnHandle = (InformationSchemaColumnHandle) entry.getKey(); filters.put(informationSchemaColumnHandle.getColumnName(), entry.getValue()); } @@ -99,11 +106,11 @@ public static class InformationSchemaPartition implements Partition { private final InformationSchemaTableHandle table; - private final Map filters; + private final Map> filters; - public InformationSchemaPartition(InformationSchemaTableHandle table, Map filters) + public InformationSchemaPartition(InformationSchemaTableHandle table, Map> filters) { - this.table = table; + this.table = checkNotNull(table, "table is null"); this.filters = ImmutableMap.copyOf(checkNotNull(filters, "filters is null")); } @@ -119,12 +126,12 @@ public String getPartitionId() } @Override - public Map getKeys() + public TupleDomain getTupleDomain() { - return ImmutableMap.of(); + return TupleDomain.withFixedValues(filters); } - public Map getFilters() + public Map> getFilters() { return filters; } diff --git a/presto-main/src/main/java/com/facebook/presto/connector/jmx/JmxSplitManager.java b/presto-main/src/main/java/com/facebook/presto/connector/jmx/JmxSplitManager.java index f2359c161957a..f67348ab63e4f 100644 --- a/presto-main/src/main/java/com/facebook/presto/connector/jmx/JmxSplitManager.java +++ b/presto-main/src/main/java/com/facebook/presto/connector/jmx/JmxSplitManager.java @@ -15,20 +15,19 @@ import com.facebook.presto.metadata.Node; import com.facebook.presto.metadata.NodeManager; -import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.ConnectorSplitManager; import com.facebook.presto.spi.Partition; +import com.facebook.presto.spi.PartitionResult; import com.facebook.presto.spi.Split; import com.facebook.presto.spi.TableHandle; +import com.facebook.presto.spi.TupleDomain; import com.google.common.base.Objects; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import javax.inject.Inject; import java.util.List; -import java.util.Map; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; @@ -59,15 +58,16 @@ public boolean canHandle(TableHandle tableHandle) } @Override - public List getPartitions(TableHandle table, Map bindings) + public PartitionResult getPartitions(TableHandle table, TupleDomain tupleDomain) { checkNotNull(table, "table is null"); - checkNotNull(bindings, "bindings is null"); + checkNotNull(tupleDomain, "tupleDomain is null"); checkArgument(table instanceof JmxTableHandle, "TableHandle must be an JmxTableHandle"); JmxTableHandle jmxTableHandle = (JmxTableHandle) table; - return ImmutableList.of(new JmxPartition(jmxTableHandle)); + ImmutableList partitions = ImmutableList.of(new JmxPartition(jmxTableHandle)); + return new PartitionResult(partitions, tupleDomain); } @Override @@ -111,9 +111,9 @@ public String getPartitionId() } @Override - public Map getKeys() + public TupleDomain getTupleDomain() { - return ImmutableMap.of(); + return TupleDomain.all(); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/connector/system/SystemSplitManager.java b/presto-main/src/main/java/com/facebook/presto/connector/system/SystemSplitManager.java index 2b874b60cc0fc..c466e0410c8d4 100644 --- a/presto-main/src/main/java/com/facebook/presto/connector/system/SystemSplitManager.java +++ b/presto-main/src/main/java/com/facebook/presto/connector/system/SystemSplitManager.java @@ -19,14 +19,17 @@ import com.facebook.presto.spi.ConnectorSplitManager; import com.facebook.presto.spi.HostAddress; import com.facebook.presto.spi.Partition; +import com.facebook.presto.spi.PartitionResult; import com.facebook.presto.spi.SchemaTableName; import com.facebook.presto.spi.Split; import com.facebook.presto.spi.SystemTable; import com.facebook.presto.spi.TableHandle; +import com.facebook.presto.spi.TupleDomain; import com.google.common.base.Objects; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; +import com.google.common.collect.Maps; import javax.inject.Inject; @@ -38,6 +41,8 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Predicates.in; +import static com.google.common.base.Predicates.not; public class SystemSplitManager implements ConnectorSplitManager @@ -72,15 +77,23 @@ public boolean canHandle(TableHandle handle) } @Override - public List getPartitions(TableHandle table, Map bindings) + public PartitionResult getPartitions(TableHandle table, TupleDomain tupleDomain) { checkNotNull(table, "table is null"); - checkNotNull(bindings, "bindings is null"); + checkNotNull(tupleDomain, "tupleDomain is null"); checkArgument(table instanceof SystemTableHandle, "TableHandle must be an SystemTableHandle"); SystemTableHandle systemTableHandle = (SystemTableHandle) table; - return ImmutableList.of(new SystemPartition(systemTableHandle, bindings)); + Map> bindings = tupleDomain.extractFixedValues(); + + TupleDomain unusedTupleDomain = TupleDomain.none(); + if (!tupleDomain.isNone()) { + unusedTupleDomain = TupleDomain.withColumnDomains(Maps.filterKeys(tupleDomain.getDomains(), not(in(bindings.keySet())))); + } + + ImmutableList partitions = ImmutableList.of(new SystemPartition(systemTableHandle, bindings)); + return new PartitionResult(partitions, unusedTupleDomain); } @Override @@ -99,7 +112,7 @@ public Iterable getPartitionSplits(TableHandle table, List par checkArgument(systemTable != null, "Table %s does not exist", systemPartition.getTableHandle().getTableName()); ImmutableMap.Builder filters = ImmutableMap.builder(); - for (Entry entry : systemPartition.getFilters().entrySet()) { + for (Entry> entry : systemPartition.getFilters().entrySet()) { SystemColumnHandle systemColumnHandle = (SystemColumnHandle) entry.getKey(); filters.put(systemColumnHandle.getColumnName(), entry.getValue()); } @@ -121,9 +134,9 @@ public static class SystemPartition implements Partition { private final SystemTableHandle tableHandle; - private final Map filters; + private final Map> filters; - public SystemPartition(SystemTableHandle tableHandle, Map filters) + public SystemPartition(SystemTableHandle tableHandle, Map> filters) { this.tableHandle = checkNotNull(tableHandle, "tableHandle is null"); this.filters = ImmutableMap.copyOf(checkNotNull(filters, "filters is null")); @@ -141,12 +154,12 @@ public String getPartitionId() } @Override - public Map getKeys() + public TupleDomain getTupleDomain() { - return ImmutableMap.of(); + return TupleDomain.withFixedValues(filters); } - public Map getFilters() + public Map> getFilters() { return filters; } diff --git a/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryExecution.java b/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryExecution.java index 5a9d66d2e5fb5..c81f278ecddaa 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryExecution.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryExecution.java @@ -38,7 +38,6 @@ import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; import io.airlift.concurrent.ThreadPoolExecutorMBean; import io.airlift.units.Duration; import org.weakref.jmx.Managed; @@ -49,7 +48,6 @@ import java.net.URI; import java.util.List; -import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ThreadPoolExecutor; @@ -217,7 +215,7 @@ private void planDistribution(SubPlan subplan) long distributedPlanningStart = System.nanoTime(); // plan the execution on the active nodes - DistributedExecutionPlanner distributedPlanner = new DistributedExecutionPlanner(splitManager, stateMachine.getSession(), shardManager); + DistributedExecutionPlanner distributedPlanner = new DistributedExecutionPlanner(splitManager, shardManager); StageExecutionPlan outputStageExecutionPlan = distributedPlanner.plan(subplan); if (stateMachine.isDone()) { diff --git a/presto-main/src/main/java/com/facebook/presto/split/NativeSplitManager.java b/presto-main/src/main/java/com/facebook/presto/split/NativeSplitManager.java index 7294fa22fc0e0..2ad21cd80ef67 100644 --- a/presto-main/src/main/java/com/facebook/presto/split/NativeSplitManager.java +++ b/presto-main/src/main/java/com/facebook/presto/split/NativeSplitManager.java @@ -22,12 +22,13 @@ import com.facebook.presto.metadata.TablePartition; import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.ConnectorSplitManager; -import com.facebook.presto.spi.ConnectorTableMetadata; import com.facebook.presto.spi.HostAddress; import com.facebook.presto.spi.Partition; import com.facebook.presto.spi.PartitionKey; +import com.facebook.presto.spi.PartitionResult; import com.facebook.presto.spi.Split; import com.facebook.presto.spi.TableHandle; +import com.facebook.presto.spi.TupleDomain; import com.google.common.base.Objects; import com.google.common.base.Stopwatch; import com.google.common.collect.Collections2; @@ -85,7 +86,7 @@ public boolean canHandle(TableHandle handle) } @Override - public List getPartitions(TableHandle tableHandle, Map bindings) + public PartitionResult getPartitions(TableHandle tableHandle, TupleDomain tupleDomain) { Stopwatch partitionTimer = new Stopwatch(); partitionTimer.start(); @@ -109,7 +110,7 @@ public List getPartitions(TableHandle tableHandle, Map keys; + private final TupleDomain tupleDomain; - public NativePartition(long partitionId, Map keys) + public NativePartition(long partitionId, TupleDomain tupleDomain) { this.partitionId = partitionId; - this.keys = keys; + this.tupleDomain = checkNotNull(tupleDomain, "tupleDomain is null"); } @Override @@ -185,15 +186,15 @@ public long getNativePartitionId() } @Override - public Map getKeys() + public TupleDomain getTupleDomain() { - return keys; + return tupleDomain; } @Override public int hashCode() { - return Objects.hashCode(partitionId, keys); + return Objects.hashCode(partitionId, tupleDomain); } @Override @@ -207,7 +208,7 @@ public boolean equals(Object obj) } final NativePartition other = (NativePartition) obj; return this.partitionId == other.partitionId - && Objects.equal(this.keys, other.keys); + && Objects.equal(this.tupleDomain, other.tupleDomain); } @Override @@ -215,7 +216,7 @@ public String toString() { return Objects.toStringHelper(this) .add("partitionId", partitionId) - .add("keys", keys) + .add("tupleDomain", tupleDomain) .toString(); } } diff --git a/presto-main/src/main/java/com/facebook/presto/split/PartitionFunction.java b/presto-main/src/main/java/com/facebook/presto/split/PartitionFunction.java index 67fe302378460..1e1d6ce375dd3 100644 --- a/presto-main/src/main/java/com/facebook/presto/split/PartitionFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/split/PartitionFunction.java @@ -15,8 +15,10 @@ import com.facebook.presto.metadata.TablePartition; import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.Domain; import com.facebook.presto.spi.Partition; import com.facebook.presto.spi.PartitionKey; +import com.facebook.presto.spi.TupleDomain; import com.facebook.presto.split.NativeSplitManager.NativePartition; import com.google.common.base.Function; import com.google.common.collect.ImmutableMap; @@ -45,7 +47,7 @@ public Partition apply(TablePartition tablePartition) { String partitionName = tablePartition.getPartitionName(); - ImmutableMap.Builder builder = ImmutableMap.builder(); + ImmutableMap.Builder builder = ImmutableMap.builder(); for (PartitionKey partitionKey : allPartitionKeys.get(partitionName)) { ColumnHandle columnHandle = columnHandles.get(partitionKey.getName()); checkArgument(columnHandles != null, "Invalid partition key for column %s in partition %s", partitionKey.getName(), tablePartition.getPartitionName()); @@ -54,34 +56,34 @@ public Partition apply(TablePartition tablePartition) switch (partitionKey.getType()) { case BOOLEAN: if (value.length() == 0) { - builder.put(columnHandle, false); + builder.put(columnHandle, Domain.singleValue(false)); } else { - builder.put(columnHandle, Boolean.parseBoolean(value)); + builder.put(columnHandle, Domain.singleValue(Boolean.parseBoolean(value))); } break; case LONG: if (value.length() == 0) { - builder.put(columnHandle, 0L); + builder.put(columnHandle, Domain.singleValue(0L)); } else { - builder.put(columnHandle, Long.parseLong(value)); + builder.put(columnHandle, Domain.singleValue(Long.parseLong(value))); } break; case DOUBLE: if (value.length() == 0) { - builder.put(columnHandle, 0L); + builder.put(columnHandle, Domain.singleValue(0.0)); } else { - builder.put(columnHandle, Double.parseDouble(value)); + builder.put(columnHandle, Domain.singleValue(Double.parseDouble(value))); } break; case STRING: - builder.put(columnHandle, value); + builder.put(columnHandle, Domain.singleValue(value)); break; } } - return new NativePartition(tablePartition.getPartitionId(), builder.build()); + return new NativePartition(tablePartition.getPartitionId(), TupleDomain.withColumnDomains(builder.build())); } } diff --git a/presto-main/src/main/java/com/facebook/presto/split/SplitManager.java b/presto-main/src/main/java/com/facebook/presto/split/SplitManager.java index 5090dcdc931f7..09a24aacb6895 100644 --- a/presto-main/src/main/java/com/facebook/presto/split/SplitManager.java +++ b/presto-main/src/main/java/com/facebook/presto/split/SplitManager.java @@ -14,49 +14,30 @@ package com.facebook.presto.split; import com.facebook.presto.execution.DataSource; -import com.facebook.presto.metadata.Metadata; import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.ConnectorSplitManager; +import com.facebook.presto.spi.Domain; import com.facebook.presto.spi.Partition; +import com.facebook.presto.spi.PartitionResult; import com.facebook.presto.spi.TableHandle; -import com.facebook.presto.sql.analyzer.Session; -import com.facebook.presto.sql.planner.ExpressionInterpreter; -import com.facebook.presto.sql.planner.LookupSymbolResolver; -import com.facebook.presto.sql.planner.Symbol; -import com.facebook.presto.sql.tree.Expression; +import com.facebook.presto.spi.TupleDomain; import com.google.common.base.Optional; -import com.google.common.base.Predicate; -import com.google.common.base.Stopwatch; -import com.google.common.collect.BiMap; -import com.google.common.collect.ImmutableBiMap; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; import com.google.inject.Inject; -import io.airlift.log.Logger; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.TimeUnit; - -import static com.facebook.presto.sql.ExpressionUtils.and; -import static com.facebook.presto.sql.ExpressionUtils.extractConjuncts; -import static com.google.common.base.Preconditions.checkNotNull; -import static com.google.common.collect.Iterables.filter; public class SplitManager { - private static final Logger log = Logger.get(SplitManager.class); - - private final Metadata metadata; private final Set splitManagers = Sets.newSetFromMap(new ConcurrentHashMap()); @Inject - public SplitManager(Metadata metadata, Set splitManagers) + public SplitManager(Set splitManagers) { - this.metadata = checkNotNull(metadata, "metadata is null"); this.splitManagers.addAll(splitManagers); } @@ -65,103 +46,18 @@ public void addConnectorSplitManager(ConnectorSplitManager connectorSplitManager splitManagers.add(connectorSplitManager); } - public DataSource getSplits(Session session, TableHandle handle, Expression predicate, Expression upstreamHint, Predicate partitionPredicate, Map mappings) + public PartitionResult getPartitions(TableHandle table, Optional tupleDomain) { - List partitions = getPartitions(session, handle, and(predicate, upstreamHint), partitionPredicate, mappings); + return getConnectorSplitManager(table).getPartitions(table, tupleDomain.or(TupleDomain.all())); + } + public DataSource getPartitionSplits(TableHandle handle, List partitions) + { ConnectorSplitManager connectorSplitManager = getConnectorSplitManager(handle); - String connectorId = connectorSplitManager.getConnectorId(); return new DataSource(connectorId, connectorSplitManager.getPartitionSplits(handle, partitions)); } - private List getPartitions(Session session, - TableHandle table, - Expression predicate, - Predicate partitionPredicate, - Map mappings) - { - Stopwatch partitionTimer = new Stopwatch(); - partitionTimer.start(); - - BiMap symbolToColumn = ImmutableBiMap.copyOf(mappings); - - // First find candidate partitions -- try to push down the predicate to the underlying API - List partitions = getCandidatePartitions(table, predicate, symbolToColumn); - - log.debug("Partition retrieval, table %s (%d partitions): %dms", table, partitions.size(), partitionTimer.elapsed(TimeUnit.MILLISECONDS)); - - // filter partitions using the specified predicate - partitions = ImmutableList.copyOf(filter(partitions, partitionPredicate)); - - log.debug("Partition filter, table %s (%d partitions): %dms", table, partitions.size(), partitionTimer.elapsed(TimeUnit.MILLISECONDS)); - - // Next, prune the list in case we got more partitions that necessary because parts of the predicate - // could not be pushed down - partitions = prunePartitions(session, partitions, predicate, symbolToColumn.inverse()); - - log.debug("Partition pruning, table %s (%d partitions): %dms", table, partitions.size(), partitionTimer.elapsed(TimeUnit.MILLISECONDS)); - - return partitions; - } - - /** - * Get candidate partitions from underlying API and make a best effort to push down any relevant parts of the provided predicate - */ - private List getCandidatePartitions(final TableHandle table, Expression predicate, Map symbolToColumnName) - { - Optional> bindings = ExpressionUtil.extractConstantValues(predicate, symbolToColumnName); - - // if bindings could not be build, no partitions will match - if (!bindings.isPresent()) { - return ImmutableList.of(); - } - - return getPartitions(table, bindings); - } - - public List getPartitions(TableHandle table, Optional> bindings) - { - checkNotNull(table, "table is null"); - return getConnectorSplitManager(table).getPartitions(table, bindings.or(ImmutableMap.of())); - } - - private List prunePartitions(Session session, List partitions, Expression predicate, Map columnToSymbol) - { - ImmutableList.Builder partitionBuilder = ImmutableList.builder(); - for (Partition partition : partitions) { - if (!shouldPrunePartition(session, partition, predicate, columnToSymbol)) { - partitionBuilder.add(partition); - } - } - return partitionBuilder.build(); - } - - private boolean shouldPrunePartition(Session session, Partition partition, Expression predicate, Map columnToSymbol) - { - // translate assignments from column->value to symbol->value - ImmutableMap.Builder assignments = ImmutableMap.builder(); - for (Map.Entry entry : partition.getKeys().entrySet()) { - ColumnHandle columnHandle = entry.getKey(); - if (columnToSymbol.containsKey(columnHandle)) { - Symbol symbol = columnToSymbol.get(columnHandle); - assignments.put(symbol, entry.getValue()); - } - } - - LookupSymbolResolver inputs = new LookupSymbolResolver(assignments.build()); - - // If any conjuncts evaluate to FALSE or null, then the whole predicate will never be true and so the partition should be pruned - for (Expression expression : extractConjuncts(predicate)) { - ExpressionInterpreter optimizer = ExpressionInterpreter.expressionOptimizer(expression, metadata, session); - Object optimized = optimizer.optimize(inputs); - if (Boolean.FALSE.equals(optimized) || optimized == null) { - return true; - } - } - return false; - } - private ConnectorSplitManager getConnectorSplitManager(TableHandle handle) { for (ConnectorSplitManager connectorSplitManager : splitManagers) { diff --git a/presto-main/src/main/java/com/facebook/presto/sql/ExpressionUtils.java b/presto-main/src/main/java/com/facebook/presto/sql/ExpressionUtils.java index 8f827b1b4d1b5..1e798f07a006a 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/ExpressionUtils.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/ExpressionUtils.java @@ -32,6 +32,7 @@ import java.util.List; import static com.facebook.presto.sql.planner.DeterminismEvaluator.deterministic; +import static com.facebook.presto.sql.tree.BooleanLiteral.FALSE_LITERAL; import static com.facebook.presto.sql.tree.BooleanLiteral.TRUE_LITERAL; import static com.google.common.base.Predicates.not; import static com.google.common.collect.Iterables.filter; @@ -51,6 +52,19 @@ public static List extractConjuncts(Expression expression) return ImmutableList.of(expression); } + public static List extractDisjuncts(Expression expression) + { + if (expression instanceof LogicalBinaryExpression && ((LogicalBinaryExpression) expression).getType() == LogicalBinaryExpression.Type.OR) { + LogicalBinaryExpression or = (LogicalBinaryExpression) expression; + return ImmutableList.builder() + .addAll(extractDisjuncts(or.getLeft())) + .addAll(extractDisjuncts(or.getRight())) + .build(); + } + + return ImmutableList.of(expression); + } + public static Expression and(Expression... expressions) { return and(Arrays.asList(expressions)); @@ -93,6 +107,11 @@ public static Expression combineConjuncts(Expression... expressions) } public static Expression combineConjuncts(Iterable expressions) + { + return combineConjunctsWithDefault(expressions, TRUE_LITERAL); + } + + public static Expression combineConjunctsWithDefault(Iterable expressions, Expression emptyDefault) { Preconditions.checkNotNull(expressions, "expressions is null"); @@ -108,15 +127,38 @@ public Iterable apply(Expression expression) // Strip out all true literal conjuncts expressions = Iterables.filter(expressions, not(Predicates.equalTo(TRUE_LITERAL))); + expressions = removeDuplicates(expressions); + return Iterables.isEmpty(expressions) ? emptyDefault : and(expressions); + } + + public static Expression combineDisjuncts(Expression... expressions) + { + return combineDisjuncts(Arrays.asList(expressions)); + } + + public static Expression combineDisjuncts(Iterable expressions) + { + return combineDisjunctsWithDefault(expressions, FALSE_LITERAL); + } - // Capture all non-deterministic conjuncts - Iterable nonDeterministicConjuncts = Iterables.filter(expressions, not(deterministic())); + public static Expression combineDisjunctsWithDefault(Iterable expressions, Expression emptyDefault) + { + Preconditions.checkNotNull(expressions, "expressions is null"); - // Capture and de-dupe all deterministic conjuncts - Iterable deterministicConjuncts = ImmutableSet.copyOf(Iterables.filter(expressions, deterministic())); + // Flatten all the expressions into their component disjuncts + expressions = Iterables.concat(Iterables.transform(expressions, new Function>() + { + @Override + public Iterable apply(Expression expression) + { + return extractDisjuncts(expression); + } + })); - expressions = Iterables.concat(nonDeterministicConjuncts, deterministicConjuncts); - return Iterables.isEmpty(expressions) ? TRUE_LITERAL : and(expressions); + // Strip out all false literal disjuncts + expressions = Iterables.filter(expressions, not(Predicates.equalTo(FALSE_LITERAL))); + expressions = removeDuplicates(expressions); + return Iterables.isEmpty(expressions) ? emptyDefault : or(expressions); } public static Function symbolToQualifiedNameReference() @@ -156,4 +198,15 @@ public Expression apply(Expression expression) } }; } + + private static Iterable removeDuplicates(Iterable expressions) + { + // Capture all non-deterministic predicates + Iterable nonDeterministicDisjuncts = Iterables.filter(expressions, not(deterministic())); + + // Capture and de-dupe all deterministic predicates + Iterable deterministicDisjuncts = ImmutableSet.copyOf(Iterables.filter(expressions, deterministic())); + + return Iterables.concat(nonDeterministicDisjuncts, deterministicDisjuncts); + } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/DistributedExecutionPlanner.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/DistributedExecutionPlanner.java index 8c0f2de6003e5..8bab116ea0ee2 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/DistributedExecutionPlanner.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/DistributedExecutionPlanner.java @@ -16,9 +16,10 @@ import com.facebook.presto.execution.DataSource; import com.facebook.presto.metadata.ShardManager; import com.facebook.presto.spi.Partition; +import com.facebook.presto.spi.PartitionResult; import com.facebook.presto.spi.Split; +import com.facebook.presto.spi.TupleDomain; import com.facebook.presto.split.SplitManager; -import com.facebook.presto.sql.analyzer.Session; import com.facebook.presto.sql.planner.plan.AggregationNode; import com.facebook.presto.sql.planner.plan.ExchangeNode; import com.facebook.presto.sql.planner.plan.FilterNode; @@ -40,6 +41,7 @@ import com.google.common.base.Optional; import com.google.common.base.Predicate; import com.google.common.base.Predicates; +import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; @@ -48,6 +50,7 @@ import javax.inject.Inject; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.ThreadLocalRandom; @@ -57,16 +60,12 @@ public class DistributedExecutionPlanner { private final SplitManager splitManager; - private final Session session; private final ShardManager shardManager; @Inject - public DistributedExecutionPlanner(SplitManager splitManager, - Session session, - ShardManager shardManager) + public DistributedExecutionPlanner(SplitManager splitManager, ShardManager shardManager) { this.splitManager = checkNotNull(splitManager, "splitManager is null"); - this.session = checkNotNull(session, "session is null"); this.shardManager = checkNotNull(shardManager, "databaseShardManager is null"); } @@ -108,17 +107,26 @@ public Map getOutputReceivers() @Override public NodeSplits visitTableScan(TableScanNode node, Predicate materializedViewPartitionPredicate) { + List partitions = FluentIterable.from(getPartitions(node)) + .filter(materializedViewPartitionPredicate) + .toList(); + // get dataSource for table - DataSource dataSource = splitManager.getSplits(session, - node.getTable(), - node.getPartitionPredicate(), - node.getUpstreamPredicateHint(), - materializedViewPartitionPredicate, - node.getAssignments()); + DataSource dataSource = splitManager.getPartitionSplits(node.getTable(), partitions); return new NodeSplits(node.getId(), dataSource); } + private List getPartitions(TableScanNode node) + { + if (node.getGeneratedPartitions().isPresent()) { + return node.getGeneratedPartitions().get().getPartitions(); + } + + PartitionResult allPartitions = splitManager.getPartitions(node.getTable(), Optional.absent()); + return allPartitions.getPartitions(); + } + @Override public NodeSplits visitJoin(JoinNode node, Predicate materializedViewPartitionPredicate) { diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/DomainTranslator.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/DomainTranslator.java new file mode 100644 index 0000000000000..5972e007449f8 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/DomainTranslator.java @@ -0,0 +1,557 @@ +/* + * 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.ColumnType; +import com.facebook.presto.spi.Domain; +import com.facebook.presto.spi.Range; +import com.facebook.presto.spi.SortedRangeSet; +import com.facebook.presto.spi.TupleDomain; +import com.facebook.presto.sql.analyzer.Type; +import com.facebook.presto.sql.tree.AstVisitor; +import com.facebook.presto.sql.tree.BetweenPredicate; +import com.facebook.presto.sql.tree.BooleanLiteral; +import com.facebook.presto.sql.tree.ComparisonExpression; +import com.facebook.presto.sql.tree.DoubleLiteral; +import com.facebook.presto.sql.tree.Expression; +import com.facebook.presto.sql.tree.InListExpression; +import com.facebook.presto.sql.tree.InPredicate; +import com.facebook.presto.sql.tree.IsNotNullPredicate; +import com.facebook.presto.sql.tree.IsNullPredicate; +import com.facebook.presto.sql.tree.Literal; +import com.facebook.presto.sql.tree.LogicalBinaryExpression; +import com.facebook.presto.sql.tree.LongLiteral; +import com.facebook.presto.sql.tree.NotExpression; +import com.facebook.presto.sql.tree.NullLiteral; +import com.facebook.presto.sql.tree.QualifiedNameReference; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.math.DoubleMath; +import io.airlift.slice.Slice; + +import java.math.RoundingMode; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static com.facebook.presto.sql.ExpressionUtils.and; +import static com.facebook.presto.sql.ExpressionUtils.combineConjuncts; +import static com.facebook.presto.sql.ExpressionUtils.combineDisjunctsWithDefault; +import static com.facebook.presto.sql.ExpressionUtils.or; +import static com.facebook.presto.sql.planner.LiteralInterpreter.toExpression; +import static com.facebook.presto.sql.tree.BooleanLiteral.FALSE_LITERAL; +import static com.facebook.presto.sql.tree.BooleanLiteral.TRUE_LITERAL; +import static com.facebook.presto.sql.tree.ComparisonExpression.Type.EQUAL; +import static com.facebook.presto.sql.tree.ComparisonExpression.Type.NOT_EQUAL; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; +import static com.google.common.collect.Iterables.getOnlyElement; + +public final class DomainTranslator +{ + private DomainTranslator() + { + } + + public static Expression toPredicate(TupleDomain tupleDomain, Map symbolTranslationMap) + { + if (tupleDomain.isNone()) { + return FALSE_LITERAL; + } + ImmutableList.Builder conjunctBuilder = ImmutableList.builder(); + for (Map.Entry entry : tupleDomain.getDomains().entrySet()) { + ColumnHandle columnHandle = entry.getKey(); + checkArgument(symbolTranslationMap.containsKey(columnHandle), "Unable to convert TupleDomain to Expression b/c don't know Symbol for ColumnHandle %s", columnHandle); + QualifiedNameReference reference = new QualifiedNameReference(symbolTranslationMap.get(columnHandle).toQualifiedName()); + conjunctBuilder.add(toPredicate(entry.getValue(), reference)); + } + return combineConjuncts(conjunctBuilder.build()); + } + + private static Expression toPredicate(Domain domain, QualifiedNameReference reference) + { + if (domain.getRanges().isNone()) { + return domain.isNullAllowed() ? new IsNullPredicate(reference) : FALSE_LITERAL; + } + + if (domain.getRanges().isAll()) { + return domain.isNullAllowed() ? TRUE_LITERAL : new IsNotNullPredicate(reference); + } + + // Add disjuncts for ranges + List disjuncts = new ArrayList<>(); + List singleValues = new ArrayList<>(); + for (Range range : domain.getRanges()) { + checkState(!range.isAll()); // Already checked + if (range.isSingleValue()) { + singleValues.add(toExpression(range.getLow().getValue())); + } + else { + List rangeConjuncts = new ArrayList<>(); + if (!range.getLow().isLowerUnbounded()) { + switch (range.getLow().getBound()) { + case ABOVE: + rangeConjuncts.add(new ComparisonExpression(ComparisonExpression.Type.GREATER_THAN, reference, toExpression(range.getLow().getValue()))); + break; + case EXACTLY: + rangeConjuncts.add(new ComparisonExpression(ComparisonExpression.Type.GREATER_THAN_OR_EQUAL, reference, toExpression(range.getLow().getValue()))); + break; + case BELOW: + throw new IllegalStateException("Low Marker should never use BELOW bound: " + range); + default: + throw new AssertionError("Unhandled bound: " + range.getLow().getBound()); + } + } + if (!range.getHigh().isUpperUnbounded()) { + switch (range.getHigh().getBound()) { + case ABOVE: + throw new IllegalStateException("High Marker should never use ABOVE bound: " + range); + case EXACTLY: + rangeConjuncts.add(new ComparisonExpression(ComparisonExpression.Type.LESS_THAN_OR_EQUAL, reference, toExpression(range.getHigh().getValue()))); + break; + case BELOW: + rangeConjuncts.add(new ComparisonExpression(ComparisonExpression.Type.LESS_THAN, reference, toExpression(range.getHigh().getValue()))); + break; + default: + throw new AssertionError("Unhandled bound: " + range.getHigh().getBound()); + } + } + // If rangeConjuncts is null, then the range was ALL, which should already have been checked for + checkState(!rangeConjuncts.isEmpty()); + disjuncts.add(combineConjuncts(rangeConjuncts)); + } + } + + // Add back all of the possible single values either as an equality or an IN predicate + if (singleValues.size() == 1) { + disjuncts.add(new ComparisonExpression(EQUAL, reference, getOnlyElement(singleValues))); + } + else if (singleValues.size() > 1) { + disjuncts.add(new InPredicate(reference, new InListExpression(singleValues))); + } + + // Add nullability disjuncts + checkState(!disjuncts.isEmpty()); + if (domain.isNullAllowed()) { + disjuncts.add(new IsNullPredicate(reference)); + } + return combineDisjunctsWithDefault(disjuncts, TRUE_LITERAL); + } + + /** + * Convert an Expression predicate into an ExtractionResult consisting of: + * 1) A successfully extracted TupleDomain + * 2) An Expression fragment which represents the part of the original Expression that will need to be re-evaluated + * after filtering with the TupleDomain. + */ + public static ExtractionResult fromPredicate(Expression predicate, Map types, Map columnHandleTranslationMap) + { + return new Visitor(types, columnHandleTranslationMap).process(predicate, false); + } + + private static class Visitor + extends AstVisitor + { + private final Map types; + private final Map columnHandles; + + private Visitor(Map types, Map columnHandles) + { + this.types = ImmutableMap.copyOf(checkNotNull(types, "types is null")); + this.columnHandles = ImmutableMap.copyOf(checkNotNull(columnHandles, "columnHandles is null")); + } + + private ColumnType checkedTypeLookup(Symbol symbol) + { + Type type = types.get(symbol); + checkArgument(type != null, "Types is missing info for symbol: %s", symbol); + return type.getColumnType(); + } + + private ColumnHandle checkedColumnHandleLookup(Symbol symbol) + { + ColumnHandle columnHandle = columnHandles.get(symbol); + checkArgument(columnHandle != null, "ColumnHandles is missing info for symbol: %s", symbol); + return columnHandle; + } + + private static SortedRangeSet complementIfNecessary(SortedRangeSet range, boolean complement) + { + return complement ? range.complement() : range; + } + + private static Domain complementIfNecessary(Domain domain, boolean complement) + { + return complement ? domain.complement() : domain; + } + + private static Expression complementIfNecessary(Expression expression, boolean complement) + { + return complement ? new NotExpression(expression) : expression; + } + + @Override + protected ExtractionResult visitExpression(Expression node, Boolean complement) + { + // If we don't know how to process this node, the default response is to say that the TupleDomain is "all" + return new ExtractionResult(TupleDomain.all(), complementIfNecessary(node, complement)); + } + + @Override + protected ExtractionResult visitLogicalBinaryExpression(LogicalBinaryExpression node, Boolean complement) + { + ExtractionResult leftResult = process(node.getLeft(), complement); + ExtractionResult rightResult = process(node.getRight(), complement); + + LogicalBinaryExpression.Type type = complement ? flipLogicalBinaryType(node.getType()) : node.getType(); + switch (type) { + case AND: + return new ExtractionResult( + leftResult.getTupleDomain().intersect(rightResult.getTupleDomain()), + combineConjuncts(leftResult.getRemainingExpression(), rightResult.getRemainingExpression())); + + case OR: + TupleDomain columnUnionedTupleDomain = leftResult.getTupleDomain().columnWiseUnion(rightResult.getTupleDomain()); + + // In most cases, the columnUnionedTupleDomain is only a superset of the actual strict union + // and so we can return the current node as the remainingExpression so that all bounds will be double checked again at execution time. + Expression remainingExpression = complementIfNecessary(node, complement); + + // However, there are a few cases where the column-wise union is actually equivalent to the strict union, so we if can detect + // some of these cases, we won't have to double check the bounds unnecessarily at execution time. + + // We can only make inferences if the remaining expressions on both side are equal and deterministic + if (leftResult.getRemainingExpression().equals(rightResult.getRemainingExpression()) && + DeterminismEvaluator.isDeterministic(leftResult.getRemainingExpression())) { + // The column-wise union is equivalent to the strict union if + // 1) If both TupleDomains consist of the same exact single column (e.g. left TupleDomain => (a > 0), right TupleDomain => (a < 10)) + // 2) If one TupleDomain is a superset of the other (e.g. left TupleDomain => (a > 0, b > 0 && b < 10), right TupleDomain => (a > 5, b = 5)) + boolean matchingSingleSymbolDomains = !leftResult.getTupleDomain().isNone() + && !rightResult.getTupleDomain().isNone() + && leftResult.getTupleDomain().getDomains().size() == 1 + && rightResult.getTupleDomain().getDomains().size() == 1 + && leftResult.getTupleDomain().getDomains().keySet().equals(rightResult.getTupleDomain().getDomains().keySet()); + boolean oneSideIsSuperSet = leftResult.getTupleDomain().contains(rightResult.getTupleDomain()) || rightResult.getTupleDomain().contains(leftResult.getTupleDomain()); + + if (matchingSingleSymbolDomains || oneSideIsSuperSet) { + remainingExpression = leftResult.getRemainingExpression(); + } + } + + return new ExtractionResult(columnUnionedTupleDomain, remainingExpression); + + default: + throw new AssertionError("Unknown type: " + node.getType()); + } + } + + private static LogicalBinaryExpression.Type flipLogicalBinaryType(LogicalBinaryExpression.Type type) + { + switch (type) { + case AND: + return LogicalBinaryExpression.Type.OR; + case OR: + return LogicalBinaryExpression.Type.AND; + default: + throw new AssertionError("Unknown type: " + type); + } + } + + @Override + protected ExtractionResult visitNotExpression(NotExpression node, Boolean complement) + { + return process(node.getValue(), !complement); + } + + @Override + protected ExtractionResult visitComparisonExpression(ComparisonExpression node, Boolean complement) + { + if (!isSimpleComparison(node)) { + return super.visitComparisonExpression(node, complement); + } + node = normalizeSimpleComparison(node); + + Symbol symbol = Symbol.fromQualifiedName(((QualifiedNameReference) node.getLeft()).getName()); + ColumnType columnType = checkedTypeLookup(symbol); + ColumnHandle columnHandle = checkedColumnHandleLookup(symbol); + Object value = LiteralInterpreter.evaluate(node.getRight()); + + // Handle the cases where implicit coercions can happen in comparisons + // TODO: how to abstract this out + if (value instanceof Double && columnType == ColumnType.LONG) { + return process(coerceDoubleToLongComparison(node), complement); + } + if (value instanceof Long && columnType == ColumnType.DOUBLE) { + value = ((Long) value).doubleValue(); + } + if (value instanceof Slice) { + // String is the expected SPI type for Slice objects + value = ((Slice) value).toStringUtf8(); + } + verifyType(columnType, value); + return createComparisonExtractionResult(node.getType(), columnHandle, columnType, objectToComparable(value), complement); + } + + private ExtractionResult createComparisonExtractionResult(ComparisonExpression.Type comparisonType, ColumnHandle columnHandle, ColumnType columnType, Comparable value, boolean complement) + { + if (value == null) { + switch (comparisonType) { + case EQUAL: + case GREATER_THAN: + case GREATER_THAN_OR_EQUAL: + case LESS_THAN: + case LESS_THAN_OR_EQUAL: + case NOT_EQUAL: + return new ExtractionResult(TupleDomain.none(), TRUE_LITERAL); + + case IS_DISTINCT_FROM: + Domain domain = complementIfNecessary(Domain.notNull(columnType.getNativeType()), complement); + return new ExtractionResult( + TupleDomain.withColumnDomains(ImmutableMap.of(columnHandle, domain)), + TRUE_LITERAL); + + default: + throw new AssertionError("Unhandled type: " + comparisonType); + } + } + + Domain domain; + switch (comparisonType) { + case EQUAL: + domain = Domain.create(complementIfNecessary(SortedRangeSet.of(Range.equal(value)), complement), false); + break; + case GREATER_THAN: + domain = Domain.create(complementIfNecessary(SortedRangeSet.of(Range.greaterThan(value)), complement), false); + break; + case GREATER_THAN_OR_EQUAL: + domain = Domain.create(complementIfNecessary(SortedRangeSet.of(Range.greaterThanOrEqual(value)), complement), false); + break; + case LESS_THAN: + domain = Domain.create(complementIfNecessary(SortedRangeSet.of(Range.lessThan(value)), complement), false); + break; + case LESS_THAN_OR_EQUAL: + domain = Domain.create(complementIfNecessary(SortedRangeSet.of(Range.lessThanOrEqual(value)), complement), false); + break; + case NOT_EQUAL: + domain = Domain.create(complementIfNecessary(SortedRangeSet.of(Range.lessThan(value), Range.greaterThan(value)), complement), false); + break; + case IS_DISTINCT_FROM: + // Need to potential complement the whole domain for IS_DISTINCT_FROM since it is null-aware + domain = complementIfNecessary(Domain.create(SortedRangeSet.of(Range.lessThan(value), Range.greaterThan(value)), true), complement); + break; + default: + throw new AssertionError("Unhandled type: " + comparisonType); + } + + return new ExtractionResult( + TupleDomain.withColumnDomains(ImmutableMap.of(columnHandle, domain)), + TRUE_LITERAL); + } + + private static void verifyType(ColumnType type, Object value) + { + checkState(value == null || type.getNativeType().isInstance(value), "Value %s is not of expected type %s", value, type); + } + + private static Comparable objectToComparable(Object value) + { + return (Comparable) value; + } + + @Override + protected ExtractionResult visitInPredicate(InPredicate node, Boolean complement) + { + if (!(node.getValue() instanceof QualifiedNameReference) || !(node.getValueList() instanceof InListExpression)) { + return super.visitInPredicate(node, complement); + } + + InListExpression valueList = (InListExpression) node.getValueList(); + checkState(!valueList.getValues().isEmpty(), "InListExpression should never be empty"); + + ImmutableList.Builder disjuncts = ImmutableList.builder(); + for (Expression expression : valueList.getValues()) { + disjuncts.add(new ComparisonExpression(EQUAL, node.getValue(), expression)); + } + return process(or(disjuncts.build()), complement); + } + + @Override + protected ExtractionResult visitBetweenPredicate(BetweenPredicate node, Boolean complement) + { + // Re-write as two comparison expressions + return process(and( + new ComparisonExpression(ComparisonExpression.Type.GREATER_THAN_OR_EQUAL, node.getValue(), node.getMin()), + new ComparisonExpression(ComparisonExpression.Type.LESS_THAN_OR_EQUAL, node.getValue(), node.getMax())), complement); + } + + @Override + protected ExtractionResult visitIsNullPredicate(IsNullPredicate node, Boolean complement) + { + if (!(node.getValue() instanceof QualifiedNameReference)) { + return super.visitIsNullPredicate(node, complement); + } + + Symbol symbol = Symbol.fromQualifiedName(((QualifiedNameReference) node.getValue()).getName()); + ColumnType columnType = checkedTypeLookup(symbol); + ColumnHandle columnHandle = checkedColumnHandleLookup(symbol); + + Domain domain = complementIfNecessary(Domain.onlyNull(columnType.getNativeType()), complement); + return new ExtractionResult( + TupleDomain.withColumnDomains(ImmutableMap.of(columnHandle, domain)), + TRUE_LITERAL); + } + + @Override + protected ExtractionResult visitIsNotNullPredicate(IsNotNullPredicate node, Boolean complement) + { + if (!(node.getValue() instanceof QualifiedNameReference)) { + return super.visitIsNotNullPredicate(node, complement); + } + + Symbol symbol = Symbol.fromQualifiedName(((QualifiedNameReference) node.getValue()).getName()); + ColumnType columnType = checkedTypeLookup(symbol); + ColumnHandle columnHandle = checkedColumnHandleLookup(symbol); + + Domain domain = complementIfNecessary(Domain.notNull(columnType.getNativeType()), complement); + return new ExtractionResult( + TupleDomain.withColumnDomains(ImmutableMap.of(columnHandle, domain)), + TRUE_LITERAL); + } + + @Override + protected ExtractionResult visitBooleanLiteral(BooleanLiteral node, Boolean complement) + { + boolean value = complement ? !node.getValue() : node.getValue(); + return new ExtractionResult(value ? TupleDomain.all() : TupleDomain.none(), TRUE_LITERAL); + } + + @Override + protected ExtractionResult visitNullLiteral(NullLiteral node, Boolean complement) + { + return new ExtractionResult(TupleDomain.none(), TRUE_LITERAL); + } + } + + private static boolean isSimpleComparison(ComparisonExpression comparison) + { + return (comparison.getLeft() instanceof QualifiedNameReference && comparison.getRight() instanceof Literal) || + (comparison.getLeft() instanceof Literal && comparison.getRight() instanceof QualifiedNameReference); + } + + /** + * Normalize a simple comparison between a QualifiedNameReference and a Literal such that the QualifiedNameReference will always be on the left and the Literal on the right. + */ + private static ComparisonExpression normalizeSimpleComparison(ComparisonExpression comparison) + { + if (comparison.getLeft() instanceof QualifiedNameReference && comparison.getRight() instanceof Literal) { + return comparison; + } + else if (comparison.getLeft() instanceof Literal && comparison.getRight() instanceof QualifiedNameReference) { + return new ComparisonExpression(flipComparisonDirection(comparison.getType()), comparison.getRight(), comparison.getLeft()); + } + else { + throw new IllegalArgumentException("ComparisonExpression not a simple literal comparison: " + comparison); + } + } + + private static ComparisonExpression.Type flipComparisonDirection(ComparisonExpression.Type type) + { + switch (type) { + case LESS_THAN_OR_EQUAL: + return ComparisonExpression.Type.GREATER_THAN_OR_EQUAL; + case LESS_THAN: + return ComparisonExpression.Type.GREATER_THAN; + case GREATER_THAN_OR_EQUAL: + return ComparisonExpression.Type.LESS_THAN_OR_EQUAL; + case GREATER_THAN: + return ComparisonExpression.Type.LESS_THAN; + default: + // The remaining types have no direction association + return type; + } + } + + private static Expression coerceDoubleToLongComparison(ComparisonExpression comparison) + { + comparison = normalizeSimpleComparison(comparison); + + checkArgument(comparison.getLeft() instanceof QualifiedNameReference, "Left must be a QualifiedNameReference"); + checkArgument(comparison.getRight() instanceof DoubleLiteral, "Right must be a DoubleLiteral"); + + QualifiedNameReference reference = (QualifiedNameReference) comparison.getLeft(); + Double value = ((DoubleLiteral) comparison.getRight()).getValue(); + + switch (comparison.getType()) { + case GREATER_THAN_OR_EQUAL: + case LESS_THAN: + return new ComparisonExpression(comparison.getType(), reference, toExpression(DoubleMath.roundToLong(value, RoundingMode.CEILING))); + + case GREATER_THAN: + case LESS_THAN_OR_EQUAL: + return new ComparisonExpression(comparison.getType(), reference, toExpression(DoubleMath.roundToLong(value, RoundingMode.FLOOR))); + + case EQUAL: + Long equalValue = DoubleMath.roundToLong(value, RoundingMode.FLOOR); + if (equalValue.doubleValue() != value) { + // Return something that is false for all non-null values + return and(new ComparisonExpression(EQUAL, reference, new LongLiteral("0")), + new ComparisonExpression(NOT_EQUAL, reference, new LongLiteral("0"))); + } + return new ComparisonExpression(comparison.getType(), reference, toExpression(equalValue)); + + case NOT_EQUAL: + Long notEqualValue = DoubleMath.roundToLong(value, RoundingMode.FLOOR); + if (notEqualValue.doubleValue() != value) { + // Return something that is true for all non-null values + return or(new ComparisonExpression(EQUAL, reference, new LongLiteral("0")), + new ComparisonExpression(NOT_EQUAL, reference, new LongLiteral("0"))); + } + return new ComparisonExpression(comparison.getType(), reference, toExpression(notEqualValue)); + + case IS_DISTINCT_FROM: + Long distinctValue = DoubleMath.roundToLong(value, RoundingMode.FLOOR); + if (distinctValue.doubleValue() != value) { + return TRUE_LITERAL; + } + return new ComparisonExpression(comparison.getType(), reference, toExpression(distinctValue)); + + default: + throw new AssertionError("Unhandled type: " + comparison.getType()); + } + } + + public static class ExtractionResult + { + private final TupleDomain tupleDomain; + private final Expression remainingExpression; + + public ExtractionResult(TupleDomain tupleDomain, Expression remainingExpression) + { + this.tupleDomain = checkNotNull(tupleDomain, "tupleDomain is null"); + this.remainingExpression = checkNotNull(remainingExpression, "remainingExpression is null"); + } + + public TupleDomain getTupleDomain() + { + return tupleDomain; + } + + public Expression getRemainingExpression() + { + return remainingExpression; + } + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/DomainUtils.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/DomainUtils.java new file mode 100644 index 0000000000000..e576a5e2dba89 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/DomainUtils.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; + +import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.Domain; +import com.facebook.presto.spi.TupleDomain; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableBiMap; +import com.google.common.collect.ImmutableMap; + +import java.util.Map; + +public final class DomainUtils +{ + private DomainUtils() + { + } + + public static Map columnHandleToSymbol(Map columnMap, Map assignments) + { + Map inverseAssignments = ImmutableBiMap.copyOf(assignments).inverse(); + Preconditions.checkArgument(inverseAssignments.keySet().containsAll(columnMap.keySet()), "assignments does not contain all required column handles"); + ImmutableMap.Builder builder = ImmutableMap.builder(); + for (Map.Entry entry : columnMap.entrySet()) { + builder.put(inverseAssignments.get(entry.getKey()), entry.getValue()); + } + return builder.build(); + } + + public static String printableTupleDomainWithSymbols(TupleDomain tupleDomain, Map assignments) + { + if (tupleDomain.isNone()) { + return "None allowed"; + } + Map symbolDomains = columnHandleToSymbol(tupleDomain.getDomains(), assignments); + return symbolDomains.toString(); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/EffectivePredicateExtractor.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/EffectivePredicateExtractor.java index 211f2b02ad1e9..ae91456402f75 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/EffectivePredicateExtractor.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/EffectivePredicateExtractor.java @@ -13,6 +13,7 @@ */ package com.facebook.presto.sql.planner; +import com.facebook.presto.spi.TupleDomain; import com.facebook.presto.sql.planner.plan.AggregationNode; import com.facebook.presto.sql.planner.plan.FilterNode; import com.facebook.presto.sql.planner.plan.JoinNode; @@ -33,6 +34,7 @@ import com.facebook.presto.sql.tree.QualifiedNameReference; import com.google.common.base.Function; import com.google.common.base.Predicate; +import com.google.common.collect.ImmutableBiMap; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Sets; @@ -48,7 +50,6 @@ import static com.facebook.presto.sql.ExpressionUtils.extractConjuncts; import static com.facebook.presto.sql.ExpressionUtils.stripNonDeterministicConjuncts; import static com.facebook.presto.sql.planner.EqualityInference.createEqualityInference; -import static com.google.common.base.Preconditions.checkState; import static com.google.common.base.Predicates.in; import static com.google.common.base.Predicates.not; import static com.google.common.collect.Iterables.filter; @@ -148,10 +149,16 @@ public Expression visitLimit(LimitNode node, Void context) @Override public Expression visitTableScan(TableScanNode node, Void context) { - // TODO: we can provide even better predicates if the metadata system is able to provide us with accurate bounds on the data sets - Expression partitionPredicate = node.getPartitionPredicate(); - checkState(DeterminismEvaluator.isDeterministic(partitionPredicate)); + if (!node.getGeneratedPartitions().isPresent()) { + return BooleanLiteral.TRUE_LITERAL; + } + // The effective predicate can be computed from the intersection of the aggregate partition TupleDomain summary (generated from Partitions) + // and the TupleDomain that was initially used to generate those Partitions. We do this because we need to select the more restrictive of the two. + // Note: the TupleDomain used to generate the partitions may contain columns/predicates that are unknown to the partition TupleDomain summary, + // but those are guaranteed to be part of a FilterNode directly above this table scan, so it's ok to include. + TupleDomain tupleDomain = node.getPartitionsDomainSummary().intersect(node.getGeneratedPartitions().get().getTupleDomainInput()); + Expression partitionPredicate = DomainTranslator.toPredicate(tupleDomain, ImmutableBiMap.copyOf(node.getAssignments()).inverse()); return pullExpressionThroughSymbols(partitionPredicate, node.getOutputSymbols()); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/ExpressionInterpreter.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/ExpressionInterpreter.java index 6b820bfaf73f6..4b34e0c947ea9 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/ExpressionInterpreter.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/ExpressionInterpreter.java @@ -28,8 +28,6 @@ import com.facebook.presto.sql.tree.CoalesceExpression; import com.facebook.presto.sql.tree.ComparisonExpression; import com.facebook.presto.sql.tree.CurrentTime; -import com.facebook.presto.sql.tree.DateLiteral; -import com.facebook.presto.sql.tree.DoubleLiteral; import com.facebook.presto.sql.tree.Expression; import com.facebook.presto.sql.tree.Extract; import com.facebook.presto.sql.tree.FunctionCall; @@ -38,40 +36,33 @@ import com.facebook.presto.sql.tree.InPredicate; import com.facebook.presto.sql.tree.Input; import com.facebook.presto.sql.tree.InputReference; -import com.facebook.presto.sql.tree.IntervalLiteral; import com.facebook.presto.sql.tree.IsNotNullPredicate; import com.facebook.presto.sql.tree.IsNullPredicate; import com.facebook.presto.sql.tree.LikePredicate; import com.facebook.presto.sql.tree.Literal; import com.facebook.presto.sql.tree.LogicalBinaryExpression; -import com.facebook.presto.sql.tree.LongLiteral; import com.facebook.presto.sql.tree.NegativeExpression; import com.facebook.presto.sql.tree.Node; import com.facebook.presto.sql.tree.NotExpression; import com.facebook.presto.sql.tree.NullIfExpression; import com.facebook.presto.sql.tree.NullLiteral; -import com.facebook.presto.sql.tree.QualifiedName; import com.facebook.presto.sql.tree.QualifiedNameReference; import com.facebook.presto.sql.tree.SearchedCaseExpression; import com.facebook.presto.sql.tree.SimpleCaseExpression; import com.facebook.presto.sql.tree.StringLiteral; -import com.facebook.presto.sql.tree.TimeLiteral; -import com.facebook.presto.sql.tree.TimestampLiteral; import com.facebook.presto.sql.tree.WhenClause; import com.facebook.presto.tuple.TupleReadable; import com.google.common.base.Charsets; -import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; import io.airlift.slice.Slice; import io.airlift.slice.Slices; import org.joni.Regex; import javax.annotation.Nullable; + import java.lang.invoke.MethodHandle; import java.util.ArrayList; import java.util.HashSet; @@ -79,7 +70,8 @@ import java.util.List; import java.util.Set; -import static com.google.common.base.Charsets.UTF_8; +import static com.facebook.presto.sql.planner.LiteralInterpreter.toExpression; +import static com.facebook.presto.sql.planner.LiteralInterpreter.toExpressions; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; @@ -224,54 +216,9 @@ protected Object visitQualifiedNameReference(QualifiedNameReference node, Object } @Override - protected Long visitLongLiteral(LongLiteral node, Object context) - { - return node.getValue(); - } - - @Override - protected Double visitDoubleLiteral(DoubleLiteral node, Object context) - { - return node.getValue(); - } - - @Override - protected Slice visitStringLiteral(StringLiteral node, Object context) - { - return node.getSlice(); - } - - @Override - protected Object visitDateLiteral(DateLiteral node, Object context) - { - return node.getUnixTime(); - } - - @Override - protected Object visitTimeLiteral(TimeLiteral node, Object context) - { - return node.getUnixTime(); - } - - @Override - protected Long visitTimestampLiteral(TimestampLiteral node, Object context) - { - return node.getUnixTime(); - } - - @Override - protected Long visitIntervalLiteral(IntervalLiteral node, Object context) - { - if (node.isYearToMonth()) { - throw new UnsupportedOperationException("Month based intervals not supported yet: " + node.getType()); - } - return node.getSeconds(); - } - - @Override - protected Object visitNullLiteral(NullLiteral node, Object context) + protected Object visitLiteral(Literal node, Object context) { - return null; + return LiteralInterpreter.evaluate(node); } @Override @@ -1005,58 +952,6 @@ private Object optimize(Node node, Object context) } } - private static List toExpressions(List objects) - { - return ImmutableList.copyOf(Lists.transform(objects, new Function() - { - public Expression apply(@Nullable Object value) - { - return toExpression(value); - } - })); - } - - public static Expression toExpression(Object object) - { - if (object instanceof Expression) { - return (Expression) object; - } - - if (object instanceof Long) { - return new LongLiteral(object.toString()); - } - - if (object instanceof Double) { - Double value = (Double) object; - if (value.isNaN()) { - return new FunctionCall(new QualifiedName("nan"), ImmutableList.of()); - } - else if (value == Double.NEGATIVE_INFINITY) { - return new NegativeExpression(new FunctionCall(new QualifiedName("infinity"), ImmutableList.of())); - } - else if (value == Double.POSITIVE_INFINITY) { - return new FunctionCall(new QualifiedName("infinity"), ImmutableList.of()); - } - else { - return new DoubleLiteral(object.toString()); - } - } - - if (object instanceof Slice) { - return new StringLiteral(((Slice) object).toString(UTF_8)); - } - - if (object instanceof Boolean) { - return new BooleanLiteral(object.toString()); - } - - if (object == null) { - return new NullLiteral(); - } - - throw new UnsupportedOperationException("not yet implemented: " + object.getClass().getName()); - } - private static Predicate isNonNullLiteralPredicate() { return new Predicate() diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/LiteralInterpreter.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/LiteralInterpreter.java new file mode 100644 index 0000000000000..6ae030053184f --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/LiteralInterpreter.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; + +import com.facebook.presto.sql.tree.AstVisitor; +import com.facebook.presto.sql.tree.BooleanLiteral; +import com.facebook.presto.sql.tree.DateLiteral; +import com.facebook.presto.sql.tree.DoubleLiteral; +import com.facebook.presto.sql.tree.Expression; +import com.facebook.presto.sql.tree.FunctionCall; +import com.facebook.presto.sql.tree.IntervalLiteral; +import com.facebook.presto.sql.tree.Literal; +import com.facebook.presto.sql.tree.LongLiteral; +import com.facebook.presto.sql.tree.NegativeExpression; +import com.facebook.presto.sql.tree.NullLiteral; +import com.facebook.presto.sql.tree.QualifiedName; +import com.facebook.presto.sql.tree.StringLiteral; +import com.facebook.presto.sql.tree.TimeLiteral; +import com.facebook.presto.sql.tree.TimestampLiteral; +import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import io.airlift.slice.Slice; + +import javax.annotation.Nullable; + +import java.util.List; + +import static com.google.common.base.Charsets.UTF_8; + +public class LiteralInterpreter +{ + public static Object evaluate(Expression node) + { + if (!(node instanceof Literal)) { + throw new IllegalArgumentException("node must be a Literal"); + } + return new LiteralVisitor().process(node, null); + } + + public static List toExpressions(List objects) + { + return ImmutableList.copyOf(Lists.transform(objects, new Function() + { + public Expression apply(@Nullable Object value) + { + return toExpression(value); + } + })); + } + + public static Expression toExpression(Object object) + { + if (object instanceof Expression) { + return (Expression) object; + } + + if (object instanceof Long) { + return new LongLiteral(object.toString()); + } + + if (object instanceof Double) { + Double value = (Double) object; + if (value.isNaN()) { + return new FunctionCall(new QualifiedName("nan"), ImmutableList.of()); + } + else if (value == Double.NEGATIVE_INFINITY) { + return new NegativeExpression(new FunctionCall(new QualifiedName("infinity"), ImmutableList.of())); + } + else if (value == Double.POSITIVE_INFINITY) { + return new FunctionCall(new QualifiedName("infinity"), ImmutableList.of()); + } + else { + return new DoubleLiteral(object.toString()); + } + } + + if (object instanceof Slice) { + return new StringLiteral(((Slice) object).toString(UTF_8)); + } + + if (object instanceof String) { + return new StringLiteral((String) object); + } + + if (object instanceof Boolean) { + return new BooleanLiteral(object.toString()); + } + + if (object == null) { + return new NullLiteral(); + } + + throw new UnsupportedOperationException("not yet implemented: " + object.getClass().getName()); + } + + private static class LiteralVisitor + extends AstVisitor + { + @Override + protected Object visitLiteral(Literal node, Void context) + { + throw new UnsupportedOperationException("Unhandled literal type: " + node); + } + + @Override + protected Object visitBooleanLiteral(BooleanLiteral node, Void context) + { + return node.getValue(); + } + + @Override + protected Long visitLongLiteral(LongLiteral node, Void context) + { + return node.getValue(); + } + + @Override + protected Double visitDoubleLiteral(DoubleLiteral node, Void context) + { + return node.getValue(); + } + + @Override + protected Slice visitStringLiteral(StringLiteral node, Void context) + { + return node.getSlice(); + } + + @Override + protected Object visitDateLiteral(DateLiteral node, Void context) + { + return node.getUnixTime(); + } + + @Override + protected Object visitTimeLiteral(TimeLiteral node, Void context) + { + return node.getUnixTime(); + } + + @Override + protected Long visitTimestampLiteral(TimestampLiteral node, Void context) + { + return node.getUnixTime(); + } + + @Override + protected Long visitIntervalLiteral(IntervalLiteral node, Void context) + { + if (node.isYearToMonth()) { + throw new UnsupportedOperationException("Month based intervals not supported yet: " + node.getType()); + } + return node.getSeconds(); + } + + @Override + protected Object visitNullLiteral(NullLiteral node, Void context) + { + return null; + } + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/LogicalPlanner.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/LogicalPlanner.java index 6ea0a33a607f1..5ef60289b0ea1 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/LogicalPlanner.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/LogicalPlanner.java @@ -48,7 +48,7 @@ import java.util.List; import java.util.Map; -import static com.facebook.presto.sql.tree.BooleanLiteral.TRUE_LITERAL; +import static com.facebook.presto.sql.planner.plan.TableScanNode.GeneratedPartitions; import static com.google.common.base.Preconditions.checkState; public class LogicalPlanner @@ -146,7 +146,7 @@ private RelationPlan createMaterializedViewWriterPlan(Analysis analysis) } ImmutableList outputSymbols = outputSymbolsBuilder.build(); - plan = new RelationPlan(new TableScanNode(idAllocator.getNextId(), sourceTableHandle, outputSymbols, inputColumnsBuilder.build(), TRUE_LITERAL, TRUE_LITERAL), new TupleDescriptor(fields.build()), outputSymbols); + plan = new RelationPlan(new TableScanNode(idAllocator.getNextId(), sourceTableHandle, outputSymbols, inputColumnsBuilder.build(), Optional.absent()), new TupleDescriptor(fields.build()), outputSymbols); targetColumnHandles = columnHandleBuilder.build(); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanOptimizersFactory.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanOptimizersFactory.java index 066a8348cfde2..9d2a40a4237f4 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanOptimizersFactory.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanOptimizersFactory.java @@ -17,6 +17,7 @@ import com.facebook.presto.metadata.Metadata; import com.facebook.presto.metadata.NodeManager; import com.facebook.presto.metadata.ShardManager; +import com.facebook.presto.split.SplitManager; import com.facebook.presto.sql.planner.optimizations.ImplementSampleAsFilter; import com.facebook.presto.sql.planner.optimizations.LimitPushDown; import com.facebook.presto.sql.planner.optimizations.MergeProjections; @@ -45,7 +46,7 @@ public class PlanOptimizersFactory private List optimizers; @Inject - public PlanOptimizersFactory(Metadata metadata) + public PlanOptimizersFactory(Metadata metadata, SplitManager splitManager) { this.metadata = checkNotNull(metadata, "metadata is null"); @@ -58,7 +59,8 @@ public PlanOptimizersFactory(Metadata metadata) new PruneRedundantProjections(), new SetFlatteningOptimizer(), new LimitPushDown(), // Run the LimitPushDown after flattening set operators to make it easier to do the set flattening - new PredicatePushDown(metadata), + new PredicatePushDown(metadata, splitManager), + new PredicatePushDown(metadata, splitManager), // Run predicate push down one more time in case we can leverage new information from generated partitions new MergeProjections(), new SimplifyExpressions(metadata), // Re-run the SimplifyExpressions to simplify any recomposed expressions from other optimizations new UnaliasSymbolReferences(), // Run again because predicate pushdown might add more projections diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanPrinter.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanPrinter.java index f33693f64032c..c41186d1d08ba 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanPrinter.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanPrinter.java @@ -14,6 +14,7 @@ package com.facebook.presto.sql.planner; import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.TupleDomain; import com.facebook.presto.sql.analyzer.Type; import com.facebook.presto.sql.planner.plan.AggregationNode; import com.facebook.presto.sql.planner.plan.ExchangeNode; @@ -52,6 +53,7 @@ import java.util.List; import java.util.Map; +import static com.facebook.presto.sql.planner.DomainUtils.printableTupleDomainWithSymbols; import static com.google.common.base.Preconditions.checkNotNull; import static java.lang.String.format; @@ -212,11 +214,14 @@ public String apply(Symbol input) @Override public Void visitTableScan(TableScanNode node, Integer indent) { - print(indent, "- TableScan[%s, partition predicate=%s, upstream predicate=%s] => [%s]", node.getTable(), node.getPartitionPredicate(), node.getUpstreamPredicateHint(), formatOutputs(node.getOutputSymbols())); + TupleDomain partitionsDomainSummary = node.getPartitionsDomainSummary(); + print(indent, "- TableScan[%s, domain=%s] => [%s]", node.getTable(), printableTupleDomainWithSymbols(partitionsDomainSummary, node.getAssignments()), formatOutputs(node.getOutputSymbols())); for (Map.Entry entry : node.getAssignments().entrySet()) { - print(indent + 2, "%s := %s", entry.getKey(), entry.getValue()); + if (node.getOutputSymbols().contains(entry.getKey()) || + (!partitionsDomainSummary.isNone() && partitionsDomainSummary.getDomains().keySet().contains(entry.getValue()))) { + print(indent + 2, "%s := %s", entry.getKey(), entry.getValue()); + } } - return null; } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanSanityChecker.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanSanityChecker.java index 2e43ab49b0ae0..d8173d2904573 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanSanityChecker.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanSanityChecker.java @@ -232,8 +232,6 @@ public Void visitTableScan(TableScanNode node, Void context) verifyUniqueId(node); Preconditions.checkArgument(node.getAssignments().keySet().containsAll(node.getOutputSymbols()), "Assignments must contain mappings for output symbols"); - Set predicateSymbols = DependencyExtractor.extractUnique(node.getPartitionPredicate()); - Preconditions.checkArgument(node.getAssignments().keySet().containsAll(predicateSymbols), "Assignments must contain mappings for all partition predicate symbols"); return null; } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/QueryPlanner.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/QueryPlanner.java index 171231e504177..7614e19882516 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/QueryPlanner.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/QueryPlanner.java @@ -65,7 +65,7 @@ import java.util.Map; import java.util.Set; -import static com.facebook.presto.sql.tree.BooleanLiteral.TRUE_LITERAL; +import static com.facebook.presto.sql.planner.plan.TableScanNode.GeneratedPartitions; import static com.facebook.presto.sql.tree.FunctionCall.argumentsGetter; import static com.facebook.presto.sql.tree.SortItem.sortKeyGetter; import static com.google.common.base.Preconditions.checkState; @@ -191,7 +191,7 @@ private RelationPlan planImplicitTable() } ImmutableMap assignments = columns.build(); - TableScanNode tableScan = new TableScanNode(idAllocator.getNextId(), table, ImmutableList.copyOf(assignments.keySet()), assignments, TRUE_LITERAL, TRUE_LITERAL); + TableScanNode tableScan = new TableScanNode(idAllocator.getNextId(), table, ImmutableList.copyOf(assignments.keySet()), assignments, Optional.absent()); return new RelationPlan(tableScan, new TupleDescriptor(), ImmutableList.of()); } 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 174d235f9d0c7..d1f9f7288da27 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 @@ -47,6 +47,7 @@ import com.facebook.presto.sql.tree.Table; import com.facebook.presto.sql.tree.TableSubquery; import com.facebook.presto.sql.tree.Union; +import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableListMultimap; @@ -59,7 +60,7 @@ import static com.facebook.presto.sql.analyzer.EquiJoinClause.leftGetter; import static com.facebook.presto.sql.analyzer.EquiJoinClause.rightGetter; -import static com.facebook.presto.sql.tree.BooleanLiteral.TRUE_LITERAL; +import static com.facebook.presto.sql.planner.plan.TableScanNode.GeneratedPartitions; import static com.google.common.base.Preconditions.checkArgument; class RelationPlanner @@ -111,7 +112,7 @@ protected RelationPlan visitTable(Table node, Void context) } ImmutableList outputSymbols = outputSymbolsBuilder.build(); - return new RelationPlan(new TableScanNode(idAllocator.getNextId(), handle, outputSymbols, columns.build(), TRUE_LITERAL, TRUE_LITERAL), descriptor, outputSymbols); + return new RelationPlan(new TableScanNode(idAllocator.getNextId(), handle, outputSymbols, columns.build(), Optional.absent()), descriptor, outputSymbols); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PredicatePushDown.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PredicatePushDown.java index fb13d505903f3..225e68a0e3275 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PredicatePushDown.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PredicatePushDown.java @@ -15,7 +15,10 @@ import com.facebook.presto.metadata.Metadata; import com.facebook.presto.spi.ColumnHandle; -import com.facebook.presto.spi.TableHandle; +import com.facebook.presto.spi.Partition; +import com.facebook.presto.spi.PartitionResult; +import com.facebook.presto.spi.TupleDomain; +import com.facebook.presto.split.SplitManager; import com.facebook.presto.sql.analyzer.Analysis; import com.facebook.presto.sql.analyzer.AnalysisContext; import com.facebook.presto.sql.analyzer.ExpressionAnalyzer; @@ -25,10 +28,14 @@ import com.facebook.presto.sql.analyzer.Type; import com.facebook.presto.sql.planner.DependencyExtractor; import com.facebook.presto.sql.planner.DeterminismEvaluator; +import com.facebook.presto.sql.planner.DomainTranslator; +import com.facebook.presto.sql.planner.DomainUtils; import com.facebook.presto.sql.planner.EffectivePredicateExtractor; import com.facebook.presto.sql.planner.EqualityInference; import com.facebook.presto.sql.planner.ExpressionInterpreter; import com.facebook.presto.sql.planner.ExpressionSymbolInliner; +import com.facebook.presto.sql.planner.LiteralInterpreter; +import com.facebook.presto.sql.planner.LookupSymbolResolver; import com.facebook.presto.sql.planner.NoOpSymbolResolver; import com.facebook.presto.sql.planner.PlanNodeIdAllocator; import com.facebook.presto.sql.planner.Symbol; @@ -51,18 +58,22 @@ import com.facebook.presto.sql.tree.Expression; import com.facebook.presto.sql.tree.ExpressionTreeRewriter; import com.facebook.presto.sql.tree.LongLiteral; +import com.facebook.presto.sql.tree.NullLiteral; import com.facebook.presto.sql.tree.QualifiedNameReference; import com.facebook.presto.util.IterableTransformer; -import com.facebook.presto.util.MapTransformer; import com.google.common.base.Function; +import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.google.common.base.Predicates; +import com.google.common.base.Stopwatch; +import com.google.common.collect.ImmutableBiMap; 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.collect.Sets; +import com.google.common.collect.Maps; +import io.airlift.log.Logger; import java.util.ArrayList; import java.util.Collection; @@ -70,6 +81,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.TimeUnit; import static com.facebook.presto.sql.ExpressionUtils.and; import static com.facebook.presto.sql.ExpressionUtils.combineConjuncts; @@ -80,6 +92,7 @@ import static com.facebook.presto.sql.planner.DeterminismEvaluator.deterministic; import static com.facebook.presto.sql.planner.DeterminismEvaluator.isDeterministic; import static com.facebook.presto.sql.planner.EqualityInference.createEqualityInference; +import static com.facebook.presto.sql.planner.plan.TableScanNode.GeneratedPartitions; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; @@ -92,11 +105,15 @@ public class PredicatePushDown extends PlanOptimizer { + private static final Logger log = Logger.get(PredicatePushDown.class); + private final Metadata metadata; + private final SplitManager splitManager; - public PredicatePushDown(Metadata metadata) + public PredicatePushDown(Metadata metadata, SplitManager splitManager) { this.metadata = checkNotNull(metadata, "metadata is null"); + this.splitManager = checkNotNull(splitManager, "splitManager is null"); } @Override @@ -107,7 +124,7 @@ public PlanNode optimize(PlanNode plan, Session session, Map types checkNotNull(types, "types is null"); checkNotNull(idAllocator, "idAllocator is null"); - return PlanRewriter.rewriteWith(new Rewriter(symbolAllocator, idAllocator, metadata, session), plan, BooleanLiteral.TRUE_LITERAL); + return PlanRewriter.rewriteWith(new Rewriter(symbolAllocator, idAllocator, metadata, splitManager, session), plan, BooleanLiteral.TRUE_LITERAL); } private static class Rewriter @@ -116,13 +133,15 @@ private static class Rewriter private final SymbolAllocator symbolAllocator; private final PlanNodeIdAllocator idAllocator; private final Metadata metadata; + private final SplitManager splitManager; private final Session session; - private Rewriter(SymbolAllocator symbolAllocator, PlanNodeIdAllocator idAllocator, Metadata metadata, Session session) + private Rewriter(SymbolAllocator symbolAllocator, PlanNodeIdAllocator idAllocator, Metadata metadata, SplitManager splitManager, Session session) { this.symbolAllocator = checkNotNull(symbolAllocator, "symbolAllocator is null"); this.idAllocator = checkNotNull(idAllocator, "idAllocator is null"); this.metadata = checkNotNull(metadata, "metadata is null"); + this.splitManager = checkNotNull(splitManager, "splitManager is null"); this.session = checkNotNull(session, "session is null"); } @@ -578,7 +597,7 @@ private Function simplifyExpressions() public Expression apply(Expression expression) { ExpressionInterpreter optimizer = ExpressionInterpreter.expressionOptimizer(expression, metadata, session); - return ExpressionInterpreter.toExpression(optimizer.optimize(NoOpSymbolResolver.INSTANCE)); + return LiteralInterpreter.toExpression(optimizer.optimize(NoOpSymbolResolver.INSTANCE)); } }; } @@ -729,65 +748,64 @@ public PlanNode rewriteSample(SampleNode node, Expression inheritedPredicate, Pl @Override public PlanNode rewriteTableScan(TableScanNode node, Expression inheritedPredicate, PlanRewriter planRewriter) { - // TODO: add predicate negotiation with connector so that we can dynamically filter out the parts of the expression that can be pre-processed - // For now, assume that connectors will only be able to pre-process predicates on partition keys, - // but also pass in the remaining predicates as a hint to the table scan (needed for certain - // connectors like InformationSchema that need to know about upstream predicates). + DomainTranslator.ExtractionResult extractionResult = DomainTranslator.fromPredicate(inheritedPredicate, symbolAllocator.getTypes(), node.getAssignments()); + Expression extractionRemainingExpression = extractionResult.getRemainingExpression(); + TupleDomain tupleDomain = extractionResult.getTupleDomain(); - Set partitionSymbols = Sets.filter(MapTransformer.of(node.getAssignments()) - .filterValues(partitionColumn(node.getTable())) - .immutableMap() - .keySet(), - in(node.getOutputSymbols())); + if (node.getGeneratedPartitions().isPresent()) { + // Add back in the TupleDomain that was used to generate the previous set of Partitions if present + // And just for kicks, throw in the domain summary too (as that can only help prune down the ranges) + // The domains should never widen between each pass. + tupleDomain = tupleDomain.intersect(node.getGeneratedPartitions().get().getTupleDomainInput()).intersect(node.getPartitionsDomainSummary()); + } - EqualityInference equalityInference = createEqualityInference(inheritedPredicate); + Stopwatch partitionTimer = Stopwatch.createStarted(); + PartitionResult matchingPartitions = splitManager.getPartitions(node.getTable(), Optional.of(tupleDomain)); + List partitions = matchingPartitions.getPartitions(); + TupleDomain undeterminedTupleDomain = matchingPartitions.getUndeterminedTupleDomain(); + log.debug("Partition retrieval, table %s (%d partitions): %dms", node.getTable(), partitions.size(), partitionTimer.elapsed(TimeUnit.MILLISECONDS)); - List partitionConjuncts = new ArrayList<>(); - List postScanConjuncts = new ArrayList<>(); + Expression unevaluatedDomainPredicate = DomainTranslator.toPredicate(undeterminedTupleDomain, ImmutableBiMap.copyOf(node.getAssignments()).inverse()); - // Strip out non-deterministic conjuncts - postScanConjuncts.addAll(ImmutableList.copyOf(filter(extractConjuncts(inheritedPredicate), not(deterministic())))); - inheritedPredicate = stripNonDeterministicConjuncts(inheritedPredicate); + // Construct the post scan predicate. Add the unevaluated TupleDomain back first since those are generally cheaper to evaluate than anything we can't extract + Expression postScanPredicate = combineConjuncts(unevaluatedDomainPredicate, extractionRemainingExpression); - // Sort non-equality predicates by those that can be applied exclusively to partition keys and those that cannot - for (Expression conjunct : EqualityInference.nonInferrableConjuncts(inheritedPredicate)) { - Expression rewrittenConjunct = equalityInference.rewriteExpression(conjunct, in(partitionSymbols)); - if (rewrittenConjunct != null) { - partitionConjuncts.add(rewrittenConjunct); - } - else { - postScanConjuncts.add(conjunct); - } - } - - // Add the equality predicates back in - EqualityInference.EqualityPartition equalityPartition = equalityInference.generateEqualitiesPartitionedBy(in(partitionSymbols)); - partitionConjuncts.addAll(equalityPartition.getScopeEqualities()); - postScanConjuncts.addAll(equalityPartition.getScopeComplementEqualities()); - postScanConjuncts.addAll(equalityPartition.getScopeStraddlingEqualities()); + // Do some early partition pruning + partitions = ImmutableList.copyOf(filter(partitions, not(shouldPrunePartition(postScanPredicate, node.getAssignments())))); + GeneratedPartitions generatedPartitions = new GeneratedPartitions(tupleDomain, partitions); PlanNode output = node; - if (!partitionConjuncts.isEmpty() || !postScanConjuncts.isEmpty()) { - // Merge the partition conjuncts, but overwrite the upstream predicate hint - if (!node.getPartitionPredicate().equals(BooleanLiteral.TRUE_LITERAL)) { - partitionConjuncts.add(node.getPartitionPredicate()); - } - output = new TableScanNode(node.getId(), node.getTable(), node.getOutputSymbols(), node.getAssignments(), combineConjuncts(partitionConjuncts), combineConjuncts(postScanConjuncts)); + if (!node.getGeneratedPartitions().equals(Optional.of(generatedPartitions))) { + output = new TableScanNode(node.getId(), node.getTable(), node.getOutputSymbols(), node.getAssignments(), Optional.of(generatedPartitions)); } - if (!postScanConjuncts.isEmpty()) { - output = new FilterNode(idAllocator.getNextId(), output, combineConjuncts(postScanConjuncts)); + if (!postScanPredicate.equals(BooleanLiteral.TRUE_LITERAL)) { + output = new FilterNode(idAllocator.getNextId(), output, postScanPredicate); } return output; } - private Predicate partitionColumn(final TableHandle tableHandle) + private Predicate shouldPrunePartition(final Expression predicate, final Map symbolToColumn) { - return new Predicate() + return new Predicate() { @Override - public boolean apply(ColumnHandle columnHandle) + public boolean apply(Partition partition) { - return metadata.getColumnMetadata(tableHandle, columnHandle).isPartitionKey(); + Map> columnFixedValueAssignments = partition.getTupleDomain().extractFixedValues(); + Map> translatableAssignments = Maps.filterKeys(columnFixedValueAssignments, in(symbolToColumn.values())); + Map> symbolFixedValueAssignments = DomainUtils.columnHandleToSymbol(translatableAssignments, symbolToColumn); + + LookupSymbolResolver inputs = new LookupSymbolResolver(ImmutableMap.copyOf(symbolFixedValueAssignments)); + + // If any conjuncts evaluate to FALSE or null, then the whole predicate will never be true and so the partition should be pruned + for (Expression expression : extractConjuncts(predicate)) { + ExpressionInterpreter optimizer = ExpressionInterpreter.expressionOptimizer(expression, metadata, session); + Object optimized = optimizer.optimize(inputs); + if (Boolean.FALSE.equals(optimized) || optimized == null || optimized instanceof NullLiteral) { + return true; + } + } + return false; } }; } 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 04c49c22840fa..edba7bb09447c 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 @@ -14,7 +14,6 @@ package com.facebook.presto.sql.planner.optimizations; import com.facebook.presto.metadata.FunctionHandle; -import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.sql.analyzer.Session; import com.facebook.presto.sql.analyzer.Type; import com.facebook.presto.sql.planner.DependencyExtractor; @@ -39,16 +38,16 @@ import com.facebook.presto.sql.planner.plan.WindowNode; import com.facebook.presto.sql.tree.Expression; import com.facebook.presto.sql.tree.FunctionCall; +import com.google.common.collect.FluentIterable; 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 com.google.common.collect.Iterables; import com.google.common.collect.ListMultimap; -import com.google.common.collect.Maps; -import com.google.common.collect.Sets; import java.util.Collection; +import java.util.List; import java.util.Map; import java.util.Set; @@ -188,7 +187,9 @@ public PlanNode rewriteWindow(WindowNode node, Set expectedOutputs, Plan @Override public PlanNode rewriteTableScan(TableScanNode node, Set expectedOutputs, PlanRewriter> planRewriter) { - Set requiredTableScanOutputs = ImmutableSet.copyOf(filter(expectedOutputs, in(node.getOutputSymbols()))); + Set requiredTableScanOutputs = FluentIterable.from(expectedOutputs) + .filter(in(node.getOutputSymbols())) + .toSet(); if (requiredTableScanOutputs.isEmpty()) { for (Symbol symbol : node.getOutputSymbols()) { if (Type.isNumeric(types.get(symbol))) { @@ -202,10 +203,10 @@ public PlanNode rewriteTableScan(TableScanNode node, Set expectedOutputs } checkState(!requiredTableScanOutputs.isEmpty()); - Set requiredSymbols = Sets.union(requiredTableScanOutputs, DependencyExtractor.extractUnique(node.getPartitionPredicate())); - Map newAssignments = Maps.filterKeys(node.getAssignments(), in(requiredSymbols)); - - return new TableScanNode(node.getId(), node.getTable(), ImmutableList.copyOf(requiredTableScanOutputs), newAssignments, node.getPartitionPredicate(), node.getUpstreamPredicateHint()); + List newOutputSymbols = FluentIterable.from(node.getOutputSymbols()) + .filter(in(requiredTableScanOutputs)) + .toList(); + return new TableScanNode(node.getId(), node.getTable(), newOutputSymbols, node.getAssignments(), node.getGeneratedPartitions()); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/SimplifyExpressions.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/SimplifyExpressions.java index fdfcc7c4704fd..d69a5ffab6f77 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/SimplifyExpressions.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/SimplifyExpressions.java @@ -17,6 +17,7 @@ import com.facebook.presto.sql.analyzer.Session; import com.facebook.presto.sql.analyzer.Type; import com.facebook.presto.sql.planner.ExpressionInterpreter; +import com.facebook.presto.sql.planner.LiteralInterpreter; import com.facebook.presto.sql.planner.NoOpSymbolResolver; import com.facebook.presto.sql.planner.PlanNodeIdAllocator; import com.facebook.presto.sql.planner.Symbol; @@ -26,7 +27,6 @@ import com.facebook.presto.sql.planner.plan.PlanNodeRewriter; import com.facebook.presto.sql.planner.plan.PlanRewriter; import com.facebook.presto.sql.planner.plan.ProjectNode; -import com.facebook.presto.sql.planner.plan.TableScanNode; import com.facebook.presto.sql.tree.BooleanLiteral; import com.facebook.presto.sql.tree.Expression; import com.google.common.base.Function; @@ -90,12 +90,6 @@ public PlanNode rewriteFilter(FilterNode node, Void context, PlanRewriter return new FilterNode(node.getId(), source, simplified); } - @Override - public PlanNode rewriteTableScan(TableScanNode node, Void context, PlanRewriter planRewriter) - { - return new TableScanNode(node.getId(), node.getTable(), node.getOutputSymbols(), node.getAssignments(), simplifyExpression(node.getPartitionPredicate()), simplifyExpression(node.getUpstreamPredicateHint())); - } - private Function simplifyExpressionFunction() { return new Function() @@ -111,7 +105,7 @@ public Expression apply(Expression input) private Expression simplifyExpression(Expression input) { ExpressionInterpreter interpreter = ExpressionInterpreter.expressionOptimizer(input, metadata, session); - return ExpressionInterpreter.toExpression(interpreter.optimize(NoOpSymbolResolver.INSTANCE)); + return LiteralInterpreter.toExpression(interpreter.optimize(NoOpSymbolResolver.INSTANCE)); } } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/TableAliasSelector.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/TableAliasSelector.java index 69750fc032fdd..4583144179d6f 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/TableAliasSelector.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/TableAliasSelector.java @@ -139,7 +139,7 @@ public PlanNode rewriteTableScan(TableScanNode node, Void context, PlanRewriter< newAssignmentsBuilder.put(assignmentEntry.getKey(), aliasedColumnHandle); } - return new TableScanNode(node.getId(), aliasTableHandle.get(), node.getOutputSymbols(), newAssignmentsBuilder.build(), node.getPartitionPredicate(), node.getUpstreamPredicateHint()); + return new TableScanNode(node.getId(), aliasTableHandle.get(), node.getOutputSymbols(), newAssignmentsBuilder.build(), node.getGeneratedPartitions()); } private boolean allNodesPresent(TableHandle tableHandle) diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/UnaliasSymbolReferences.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/UnaliasSymbolReferences.java index 01342c366d691..23268d8397b42 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/UnaliasSymbolReferences.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/UnaliasSymbolReferences.java @@ -157,7 +157,7 @@ public PlanNode rewriteTableScan(TableScanNode node, Void context, PlanRewriter< builder.put(canonicalize(entry.getKey()), entry.getValue()); } - return new TableScanNode(node.getId(), node.getTable(), canonicalize(node.getOutputSymbols()), builder.build(), canonicalize(node.getPartitionPredicate()), canonicalize(node.getUpstreamPredicateHint())); + return new TableScanNode(node.getId(), node.getTable(), canonicalize(node.getOutputSymbols()), builder.build(), node.getGeneratedPartitions()); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/TableScanNode.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/TableScanNode.java index 35dc7fd1c8f58..91a592587295d 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/TableScanNode.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/TableScanNode.java @@ -14,13 +14,15 @@ package com.facebook.presto.sql.planner.plan; import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.Partition; import com.facebook.presto.spi.TableHandle; -import com.facebook.presto.sql.planner.DependencyExtractor; +import com.facebook.presto.spi.TupleDomain; +import com.facebook.presto.sql.planner.DomainUtils; import com.facebook.presto.sql.planner.Symbol; -import com.facebook.presto.sql.tree.Expression; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; +import com.google.common.base.Objects; +import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -29,6 +31,10 @@ import java.util.List; import java.util.Map; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; + @Immutable public class TableScanNode extends PlanNode @@ -36,36 +42,42 @@ public class TableScanNode private final TableHandle table; private final List outputSymbols; private final Map assignments; // symbol -> column - private final Expression partitionPredicate; - private final Expression upstreamPredicateHint; // TODO: hack to support lack of connector predicate negotiation (fix this) + private final Optional generatedPartitions; + private final boolean partitionsDroppedBySerialization; + private final TupleDomain partitionDomainSummary; + + public TableScanNode(PlanNodeId id, TableHandle table, List outputSymbols, Map assignments, Optional generatedPartitions) + { + this(id, table, outputSymbols, assignments, generatedPartitions, false); + } @JsonCreator public TableScanNode(@JsonProperty("id") PlanNodeId id, @JsonProperty("table") TableHandle table, @JsonProperty("outputSymbols") List outputSymbols, - @JsonProperty("assignments") Map assignments, - @JsonProperty("partitionPredicate") Expression partitionPredicate, - @JsonProperty("upstreamPredicateHint") Expression upstreamPredicateHint) + @JsonProperty("assignments") Map assignments) + { + this(id, table, outputSymbols, assignments, Optional.absent(), true); + } + + private TableScanNode(PlanNodeId id, TableHandle table, List outputSymbols, Map assignments, Optional generatedPartitions, boolean partitionsDroppedBySerialization) { super(id); - Preconditions.checkNotNull(table, "table is null"); - Preconditions.checkNotNull(outputSymbols, "outputSymbols is null"); - Preconditions.checkArgument(!outputSymbols.isEmpty(), "outputSymbols is empty"); - Preconditions.checkNotNull(assignments, "assignments is null"); - Preconditions.checkArgument(!assignments.isEmpty(), "assignments is empty"); - Preconditions.checkNotNull(partitionPredicate, "partitionPredicate is null"); - Preconditions.checkNotNull(upstreamPredicateHint, "upstreamPredicateHint is null"); + checkNotNull(table, "table is null"); + checkNotNull(outputSymbols, "outputSymbols is null"); + checkNotNull(assignments, "assignments is null"); + checkArgument(assignments.keySet().containsAll(outputSymbols), "assignments does not cover all of outputSymbols"); + checkArgument(!assignments.isEmpty(), "assignments is empty"); + checkNotNull(generatedPartitions, "generatedPartitions is null"); this.table = table; this.outputSymbols = ImmutableList.copyOf(outputSymbols); this.assignments = ImmutableMap.copyOf(assignments); - this.partitionPredicate = partitionPredicate; - this.upstreamPredicateHint = upstreamPredicateHint; - - Preconditions.checkArgument(assignments.keySet().containsAll(outputSymbols), "Assignments must provide mappings for all output symbols"); - Preconditions.checkArgument(assignments.keySet().containsAll(DependencyExtractor.extractUnique(partitionPredicate)), "Assignments must provide mappings for all partition predicate symbols"); - Preconditions.checkArgument(outputSymbols.containsAll(DependencyExtractor.extractUnique(upstreamPredicateHint)), "Upstream predicate hint must be in terms of output symbols"); + this.generatedPartitions = generatedPartitions; + this.partitionsDroppedBySerialization = partitionsDroppedBySerialization; + this.partitionDomainSummary = computePartitionsDomainSummary(generatedPartitions); + checkArgument(partitionDomainSummary.isNone() || assignments.values().containsAll(partitionDomainSummary.getDomains().keySet()), "Assignments do not include all of the ColumnHandles specified by the Partitions"); } @JsonProperty("table") @@ -74,28 +86,49 @@ public TableHandle getTable() return table; } + @JsonProperty("outputSymbols") + public List getOutputSymbols() + { + return outputSymbols; + } + @JsonProperty("assignments") public Map getAssignments() { return assignments; } - @JsonProperty("partitionPredicate") - public Expression getPartitionPredicate() + public Optional getGeneratedPartitions() { - return partitionPredicate; + // If this exception throws, then we might want to consider making Partitions serializable by Jackson + checkState(!partitionsDroppedBySerialization, "Can't access partitions after passing through serialization"); + return generatedPartitions; } - @JsonProperty("upstreamPredicateHint") - public Expression getUpstreamPredicateHint() + public TupleDomain getPartitionsDomainSummary() { - return upstreamPredicateHint; + return partitionDomainSummary; } - @JsonProperty("outputSymbols") - public List getOutputSymbols() + private static TupleDomain computePartitionsDomainSummary(Optional generatedPartitions) { - return outputSymbols; + if (!generatedPartitions.isPresent()) { + return TupleDomain.all(); + } + + TupleDomain tupleDomain = TupleDomain.none(); + for (Partition partition : generatedPartitions.get().getPartitions()) { + tupleDomain = tupleDomain.columnWiseUnion(partition.getTupleDomain()); + } + return tupleDomain; + } + + @JsonProperty("partitionDomainSummary") + public String getPrintablePartitionDomainSummary() + { + // Since partitions are not serializable, we can provide an additional jackson field purely for information purposes (i.e. for logging) + // If partitions ever become serializable, we can get rid of this method + return DomainUtils.printableTupleDomainWithSymbols(partitionDomainSummary, assignments); } public List getSources() @@ -107,4 +140,45 @@ public R accept(PlanVisitor visitor, C context) { return visitor.visitTableScan(this, context); } + + public final static class GeneratedPartitions + { + private final TupleDomain tupleDomainInput; // The TupleDomain used to generate the current list of Partitions + private final List partitions; + + public GeneratedPartitions(TupleDomain tupleDomainInput, List partitions) + { + this.tupleDomainInput = checkNotNull(tupleDomainInput, "tupleDomainInput is null"); + this.partitions = ImmutableList.copyOf(checkNotNull(partitions, "partitions is null")); + } + + public TupleDomain getTupleDomainInput() + { + return tupleDomainInput; + } + + public List getPartitions() + { + return partitions; + } + + @Override + public int hashCode() + { + return Objects.hashCode(tupleDomainInput, partitions); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final GeneratedPartitions other = (GeneratedPartitions) obj; + return Objects.equal(this.tupleDomainInput, other.tupleDomainInput) && Objects.equal(this.partitions, other.partitions); + } + } } diff --git a/presto-main/src/main/java/com/facebook/presto/tpch/TpchSplitManager.java b/presto-main/src/main/java/com/facebook/presto/tpch/TpchSplitManager.java index e54ea823094cb..8ab1df4927cc6 100644 --- a/presto-main/src/main/java/com/facebook/presto/tpch/TpchSplitManager.java +++ b/presto-main/src/main/java/com/facebook/presto/tpch/TpchSplitManager.java @@ -15,20 +15,19 @@ import com.facebook.presto.metadata.Node; import com.facebook.presto.metadata.NodeManager; -import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.ConnectorSplitManager; import com.facebook.presto.spi.Partition; +import com.facebook.presto.spi.PartitionResult; import com.facebook.presto.spi.Split; import com.facebook.presto.spi.TableHandle; +import com.facebook.presto.spi.TupleDomain; import com.google.common.base.Objects; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import javax.inject.Inject; import java.util.List; -import java.util.Map; import java.util.Set; import static com.google.common.base.Preconditions.checkArgument; @@ -60,9 +59,10 @@ public boolean canHandle(TableHandle handle) } @Override - public List getPartitions(TableHandle table, Map bindings) + public PartitionResult getPartitions(TableHandle table, TupleDomain tupleDomain) { - return ImmutableList.of(new TpchPartition((TpchTableHandle) table)); + ImmutableList partitions = ImmutableList.of(new TpchPartition((TpchTableHandle) table)); + return new PartitionResult(partitions, tupleDomain); } @Override @@ -98,7 +98,7 @@ public static class TpchPartition public TpchPartition(TpchTableHandle table) { - this.table = table; + this.table = checkNotNull(table, "table is null"); } public TpchTableHandle getTable() @@ -113,9 +113,9 @@ public String getPartitionId() } @Override - public Map getKeys() + public TupleDomain getTupleDomain() { - return ImmutableMap.of(); + return TupleDomain.all(); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/util/GraphvizPrinter.java b/presto-main/src/main/java/com/facebook/presto/util/GraphvizPrinter.java index 8f2d5b7c54a3b..7bfe0f0df675a 100644 --- a/presto-main/src/main/java/com/facebook/presto/util/GraphvizPrinter.java +++ b/presto-main/src/main/java/com/facebook/presto/util/GraphvizPrinter.java @@ -51,6 +51,7 @@ import java.util.List; import java.util.Map; +import static com.facebook.presto.sql.planner.DomainUtils.printableTupleDomainWithSymbols; import static com.google.common.collect.Maps.immutableEnumMap; import static java.lang.String.format; @@ -303,7 +304,7 @@ public Void visitLimit(LimitNode node, Void context) @Override public Void visitTableScan(TableScanNode node, Void context) { - printNode(node, format("TableScan[%s]", node.getTable()), format("partition predicate=%s", node.getPartitionPredicate()), NODE_COLORS.get(NodeType.TABLESCAN)); + printNode(node, format("TableScan[%s]", node.getTable()), format("domain=%s", printableTupleDomainWithSymbols(node.getPartitionsDomainSummary(), node.getAssignments())), NODE_COLORS.get(NodeType.TABLESCAN)); return null; } diff --git a/presto-main/src/test/java/com/facebook/presto/AbstractTestQueries.java b/presto-main/src/test/java/com/facebook/presto/AbstractTestQueries.java index 237a72d67d26b..99ddff408d703 100644 --- a/presto-main/src/test/java/com/facebook/presto/AbstractTestQueries.java +++ b/presto-main/src/test/java/com/facebook/presto/AbstractTestQueries.java @@ -14,12 +14,16 @@ package com.facebook.presto; import com.facebook.presto.connector.dual.DualMetadata; +import com.facebook.presto.connector.dual.DualSplitManager; import com.facebook.presto.importer.MockPeriodicImportManager; +import com.facebook.presto.metadata.InMemoryNodeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.spi.ColumnMetadata; +import com.facebook.presto.spi.ConnectorSplitManager; import com.facebook.presto.spi.ConnectorTableMetadata; import com.facebook.presto.spi.RecordCursor; import com.facebook.presto.spi.RecordSet; +import com.facebook.presto.split.SplitManager; import com.facebook.presto.sql.analyzer.QueryExplainer; import com.facebook.presto.sql.analyzer.Session; import com.facebook.presto.sql.parser.SqlParser; @@ -2870,7 +2874,8 @@ private static QueryExplainer getQueryExplainer() Session session = new Session("user", "test", DEFAULT_CATALOG, DEFAULT_SCHEMA, null, null); MetadataManager metadata = new MetadataManager(); metadata.addInternalSchemaMetadata(MetadataManager.INTERNAL_CONNECTOR_ID, new DualMetadata()); - List optimizers = new PlanOptimizersFactory(metadata).get(); + SplitManager splitManager = new SplitManager(ImmutableSet.of(new DualSplitManager(new InMemoryNodeManager()))); + List optimizers = new PlanOptimizersFactory(metadata, splitManager).get(); return new QueryExplainer(session, optimizers, metadata, new MockPeriodicImportManager(), new MockStorageManager()); } } diff --git a/presto-main/src/test/java/com/facebook/presto/execution/TestSqlStageExecution.java b/presto-main/src/test/java/com/facebook/presto/execution/TestSqlStageExecution.java index d4e8b79ec6a5d..b7b8971f666fb 100644 --- a/presto-main/src/test/java/com/facebook/presto/execution/TestSqlStageExecution.java +++ b/presto-main/src/test/java/com/facebook/presto/execution/TestSqlStageExecution.java @@ -41,7 +41,6 @@ import com.facebook.presto.sql.planner.plan.PlanFragmentId; import com.facebook.presto.sql.planner.plan.PlanNodeId; import com.facebook.presto.sql.planner.plan.TableScanNode; -import com.facebook.presto.sql.tree.BooleanLiteral; import com.facebook.presto.util.Threads; import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; @@ -70,6 +69,7 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicLong; +import static com.facebook.presto.sql.planner.plan.TableScanNode.GeneratedPartitions; import static com.facebook.presto.util.Failures.toFailures; import static com.google.common.base.Preconditions.checkNotNull; import static io.airlift.units.DataSize.Unit.MEGABYTE; @@ -189,11 +189,7 @@ private StageExecutionPlan createTableScanPlan(String planId, MetadataManager me PlanFragment testFragment = new PlanFragment(new PlanFragmentId(planId), tableScanNodeId, ImmutableMap.of(symbol, Type.VARCHAR), - new TableScanNode(tableScanNodeId, - tableHandle, ImmutableList.of(symbol), - ImmutableMap.of(symbol, columnHandle), - BooleanLiteral.TRUE_LITERAL, - BooleanLiteral.TRUE_LITERAL)); + new TableScanNode(tableScanNodeId, tableHandle, ImmutableList.of(symbol), ImmutableMap.of(symbol, columnHandle), Optional.absent())); DataSource dataSource = new DataSource(null, ImmutableList.copyOf(Collections.nCopies(splitCount, split))); return new StageExecutionPlan(testFragment, diff --git a/presto-main/src/test/java/com/facebook/presto/execution/TestSqlTaskManager.java b/presto-main/src/test/java/com/facebook/presto/execution/TestSqlTaskManager.java index 3a790f3152f1c..0157149ecc8ed 100644 --- a/presto-main/src/test/java/com/facebook/presto/execution/TestSqlTaskManager.java +++ b/presto-main/src/test/java/com/facebook/presto/execution/TestSqlTaskManager.java @@ -26,9 +26,12 @@ import com.facebook.presto.metadata.Node; import com.facebook.presto.operator.ExchangeClient; import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.Domain; +import com.facebook.presto.spi.PartitionResult; import com.facebook.presto.spi.SchemaTableName; import com.facebook.presto.spi.Split; import com.facebook.presto.spi.TableHandle; +import com.facebook.presto.spi.TupleDomain; import com.facebook.presto.split.DataStreamManager; import com.facebook.presto.sql.analyzer.Session; import com.facebook.presto.sql.analyzer.Type; @@ -39,6 +42,7 @@ import com.facebook.presto.sql.planner.plan.PlanFragmentId; import com.facebook.presto.sql.planner.plan.PlanNodeId; import com.facebook.presto.sql.planner.plan.TableScanNode; +import com.google.common.base.Optional; import com.google.common.base.Supplier; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -59,7 +63,7 @@ import java.util.NoSuchElementException; import java.util.concurrent.TimeUnit; -import static com.facebook.presto.sql.tree.BooleanLiteral.TRUE_LITERAL; +import static com.facebook.presto.sql.planner.plan.TableScanNode.GeneratedPartitions; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; @@ -95,7 +99,8 @@ public void setUp() metadata.addInternalSchemaMetadata(MetadataManager.INTERNAL_CONNECTOR_ID, dualMetadata); DualSplitManager dualSplitManager = new DualSplitManager(new InMemoryNodeManager()); - split = Iterables.getOnlyElement(dualSplitManager.getPartitionSplits(tableHandle, dualSplitManager.getPartitions(tableHandle, ImmutableMap.of()))); + PartitionResult partitionResult = dualSplitManager.getPartitions(tableHandle, TupleDomain.all()); + split = Iterables.getOnlyElement(dualSplitManager.getPartitionSplits(tableHandle, partitionResult.getPartitions())); planner = new LocalExecutionPlanner( new NodeInfo("test"), @@ -119,7 +124,7 @@ public void setUp() testFragment = new PlanFragment(new PlanFragmentId("fragment"), tableScanNodeId, ImmutableMap.of(symbol, Type.VARCHAR), - new TableScanNode(tableScanNodeId, tableHandle, ImmutableList.of(symbol), ImmutableMap.of(symbol, columnHandle), TRUE_LITERAL, TRUE_LITERAL)); + new TableScanNode(tableScanNodeId, tableHandle, ImmutableList.of(symbol), ImmutableMap.of(symbol, columnHandle), Optional.absent())); taskId = new TaskId("query", "stage", "task"); session = new Session("user", "test", "default", "default", "test", "test"); diff --git a/presto-main/src/test/java/com/facebook/presto/split/TestNativeSplitManager.java b/presto-main/src/test/java/com/facebook/presto/split/TestNativeSplitManager.java index 9119b4b48c7e2..d2e8e8b0c04ce 100644 --- a/presto-main/src/test/java/com/facebook/presto/split/TestNativeSplitManager.java +++ b/presto-main/src/test/java/com/facebook/presto/split/TestNativeSplitManager.java @@ -13,7 +13,6 @@ */ package com.facebook.presto.split; -import com.facebook.presto.execution.DataSource; import com.facebook.presto.metadata.DatabaseShardManager; import com.facebook.presto.metadata.InMemoryNodeManager; import com.facebook.presto.metadata.MetadataManager; @@ -25,26 +24,19 @@ import com.facebook.presto.metadata.TableMetadata; import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.ColumnType; -import com.facebook.presto.spi.ConnectorSplitManager; import com.facebook.presto.spi.ConnectorTableMetadata; +import com.facebook.presto.spi.Domain; import com.facebook.presto.spi.Partition; import com.facebook.presto.spi.PartitionKey; +import com.facebook.presto.spi.PartitionResult; +import com.facebook.presto.spi.Range; +import com.facebook.presto.spi.SortedRangeSet; import com.facebook.presto.spi.Split; import com.facebook.presto.spi.TableHandle; -import com.facebook.presto.sql.analyzer.Session; -import com.facebook.presto.sql.planner.Symbol; -import com.facebook.presto.sql.tree.BooleanLiteral; -import com.facebook.presto.sql.tree.ComparisonExpression; -import com.facebook.presto.sql.tree.ComparisonExpression.Type; -import com.facebook.presto.sql.tree.Expression; -import com.facebook.presto.sql.tree.QualifiedName; -import com.facebook.presto.sql.tree.QualifiedNameReference; -import com.facebook.presto.sql.tree.StringLiteral; -import com.google.common.base.Optional; -import com.google.common.base.Predicates; +import com.facebook.presto.spi.TupleDomain; 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.io.Files; import io.airlift.testing.FileUtils; import org.skife.jdbi.v2.DBI; @@ -57,14 +49,12 @@ import java.io.File; import java.net.URI; import java.util.List; -import java.util.Map; import java.util.UUID; import static com.facebook.presto.spi.ColumnType.LONG; import static com.facebook.presto.spi.ColumnType.STRING; -import static com.facebook.presto.sql.analyzer.Session.DEFAULT_CATALOG; -import static com.facebook.presto.sql.analyzer.Session.DEFAULT_SCHEMA; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; public class TestNativeSplitManager { @@ -74,16 +64,11 @@ public class TestNativeSplitManager .column("bar", LONG) .build(); - private static final Session session = new Session("user", "test", DEFAULT_CATALOG, DEFAULT_SCHEMA, null, null, System.currentTimeMillis()); - private Handle dummyHandle; private File dataDir; private NativeSplitManager nativeSplitManager; - private SplitManager splitManager; private TableHandle tableHandle; private ColumnHandle dsColumnHandle; - private ColumnHandle fooColumnHandle; - private Map symbols; @BeforeMethod public void setup() @@ -103,8 +88,6 @@ public void setup() tableHandle = metadataManager.createTable("local", new TableMetadata("local", TEST_TABLE)); dsColumnHandle = metadataManager.getColumnHandle(tableHandle, "ds").get(); - fooColumnHandle = metadataManager.getColumnHandle(tableHandle, "foo").get(); - symbols = ImmutableMap.of(new Symbol("foo"), fooColumnHandle, new Symbol("ds"), dsColumnHandle); long shardId1 = shardManager.allocateShard(tableHandle); long shardId2 = shardManager.allocateShard(tableHandle); @@ -117,7 +100,6 @@ public void setup() shardManager.commitPartition(tableHandle, "ds=2", ImmutableList.of(new NativePartitionKey("ds=2", "ds", ColumnType.STRING, "2")), ImmutableMap.of(shardId4, nodeName)); nativeSplitManager = new NativeSplitManager(nodeManager, shardManager, metadataManager); - splitManager = new SplitManager(metadataManager, ImmutableSet.of(nativeSplitManager)); } @AfterMethod @@ -128,55 +110,17 @@ public void teardown() } @Test - public void testNoPruning() - { - List partitions = splitManager.getPartitions(tableHandle, Optional.>of(ImmutableMap.of())); - assertEquals(partitions.size(), 2); - - DataSource dataSource = splitManager.getSplits(session, tableHandle, BooleanLiteral.TRUE_LITERAL, BooleanLiteral.TRUE_LITERAL, Predicates.alwaysTrue(), ImmutableMap.of(new Symbol("ds"), dsColumnHandle)); - List splits = ImmutableList.copyOf(dataSource.getSplits()); - assertEquals(splits.size(), 4); - } - - @Test - public void testPruneNoMatch() + public void testSanity() { - List partitions = splitManager.getPartitions(tableHandle, Optional.>of(ImmutableMap.of(dsColumnHandle, "foo"))); - assertEquals(partitions.size(), 2); - - // ds=3. No partition will match this. - Expression nonMatching = new ComparisonExpression(Type.EQUAL, new QualifiedNameReference(new QualifiedName("ds")), new StringLiteral("3")); - DataSource dataSource = splitManager.getSplits(session, tableHandle, BooleanLiteral.TRUE_LITERAL, nonMatching, Predicates.alwaysTrue(), symbols); - List splits = ImmutableList.copyOf(dataSource.getSplits()); - // no splits found - assertEquals(splits.size(), 0); - } + PartitionResult partitionResult = nativeSplitManager.getPartitions(tableHandle, TupleDomain.all()); + assertEquals(partitionResult.getPartitions().size(), 2); + assertTrue(partitionResult.getUndeterminedTupleDomain().isAll()); - @Test - public void testPruneMatch() - { - List partitions = splitManager.getPartitions(tableHandle, Optional.>of(ImmutableMap.of(dsColumnHandle, "1"))); - assertEquals(partitions.size(), 2); - - // ds=1. One partition with three splits will match this. - Expression nonMatching = new ComparisonExpression(Type.EQUAL, new QualifiedNameReference(new QualifiedName("ds")), new StringLiteral("1")); - DataSource dataSource = splitManager.getSplits(session, tableHandle, BooleanLiteral.TRUE_LITERAL, nonMatching, Predicates.alwaysTrue(), symbols); - List splits = ImmutableList.copyOf(dataSource.getSplits()); - // three splits found - assertEquals(splits.size(), 3); - } + List partitions = partitionResult.getPartitions(); + TupleDomain columnUnionedTupleDomain = partitions.get(0).getTupleDomain().columnWiseUnion(partitions.get(1).getTupleDomain()); + assertEquals(columnUnionedTupleDomain, TupleDomain.withColumnDomains(ImmutableMap.of(dsColumnHandle, Domain.create(SortedRangeSet.of(Range.equal("1"), Range.equal("2")), false)))); - @Test - public void testNoPruneUnknown() - { - List partitions = splitManager.getPartitions(tableHandle, Optional.>of(ImmutableMap.of(dsColumnHandle, "foo"))); - assertEquals(partitions.size(), 2); - - // foo=bar. Not a prunable column - Expression nonMatching = new ComparisonExpression(Type.EQUAL, new QualifiedNameReference(new QualifiedName("foo")), new StringLiteral("bar")); - DataSource dataSource = splitManager.getSplits(session, tableHandle, BooleanLiteral.TRUE_LITERAL, nonMatching, Predicates.alwaysTrue(), symbols); - List splits = ImmutableList.copyOf(dataSource.getSplits()); - // all splits found - assertEquals(splits.size(), 4); + Iterable splits = nativeSplitManager.getPartitionSplits(tableHandle, partitions); + assertEquals(Iterables.size(splits), 4); } } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestDomainTranslator.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestDomainTranslator.java new file mode 100644 index 0000000000000..5dfc2937c3810 --- /dev/null +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestDomainTranslator.java @@ -0,0 +1,1020 @@ +/* + * 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.Domain; +import com.facebook.presto.spi.Range; +import com.facebook.presto.spi.SortedRangeSet; +import com.facebook.presto.spi.TupleDomain; +import com.facebook.presto.sql.analyzer.Type; +import com.facebook.presto.sql.tree.BetweenPredicate; +import com.facebook.presto.sql.tree.ComparisonExpression; +import com.facebook.presto.sql.tree.DoubleLiteral; +import com.facebook.presto.sql.tree.Expression; +import com.facebook.presto.sql.tree.FunctionCall; +import com.facebook.presto.sql.tree.InListExpression; +import com.facebook.presto.sql.tree.InPredicate; +import com.facebook.presto.sql.tree.IsNotNullPredicate; +import com.facebook.presto.sql.tree.IsNullPredicate; +import com.facebook.presto.sql.tree.LongLiteral; +import com.facebook.presto.sql.tree.NotExpression; +import com.facebook.presto.sql.tree.NullLiteral; +import com.facebook.presto.sql.tree.QualifiedName; +import com.facebook.presto.sql.tree.QualifiedNameReference; +import com.facebook.presto.sql.tree.StringLiteral; +import com.google.common.collect.BiMap; +import com.google.common.collect.ImmutableBiMap; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.testng.Assert; +import org.testng.annotations.Test; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import static com.facebook.presto.spi.TupleDomain.withColumnDomains; +import static com.facebook.presto.sql.ExpressionUtils.and; +import static com.facebook.presto.sql.ExpressionUtils.or; +import static com.facebook.presto.sql.planner.DomainTranslator.ExtractionResult; +import static com.facebook.presto.sql.planner.DomainTranslator.fromPredicate; +import static com.facebook.presto.sql.planner.DomainTranslator.toPredicate; +import static com.facebook.presto.sql.tree.BooleanLiteral.FALSE_LITERAL; +import static com.facebook.presto.sql.tree.BooleanLiteral.TRUE_LITERAL; +import static com.facebook.presto.sql.tree.ComparisonExpression.Type.EQUAL; +import static com.facebook.presto.sql.tree.ComparisonExpression.Type.GREATER_THAN; +import static com.facebook.presto.sql.tree.ComparisonExpression.Type.GREATER_THAN_OR_EQUAL; +import static com.facebook.presto.sql.tree.ComparisonExpression.Type.IS_DISTINCT_FROM; +import static com.facebook.presto.sql.tree.ComparisonExpression.Type.LESS_THAN; +import static com.facebook.presto.sql.tree.ComparisonExpression.Type.LESS_THAN_OR_EQUAL; +import static com.facebook.presto.sql.tree.ComparisonExpression.Type.NOT_EQUAL; + +public class TestDomainTranslator +{ + private static final Symbol A = new Symbol("a"); + private static final ColumnHandle ACH = new TestingColumnHandle(A); + private static final Symbol B = new Symbol("b"); + private static final ColumnHandle BCH = new TestingColumnHandle(B); + private static final Symbol C = new Symbol("c"); + private static final ColumnHandle CCH = new TestingColumnHandle(C); + private static final Symbol D = new Symbol("d"); + private static final ColumnHandle DCH = new TestingColumnHandle(D); + private static final Symbol E = new Symbol("e"); + private static final ColumnHandle ECH = new TestingColumnHandle(E); + private static final Symbol F = new Symbol("f"); + private static final ColumnHandle FCH = new TestingColumnHandle(F); + private static final Symbol G = new Symbol("g"); + private static final ColumnHandle GCH = new TestingColumnHandle(G); + + private static final Map TYPES = ImmutableMap.builder() + .put(A, Type.BIGINT) + .put(B, Type.DOUBLE) + .put(C, Type.VARCHAR) + .put(D, Type.BOOLEAN) + .put(E, Type.BIGINT) + .put(F, Type.DOUBLE) + .put(G, Type.VARCHAR) + .build(); + + private static final BiMap COLUMN_HANDLES = ImmutableBiMap.builder() + .put(A, ACH) + .put(B, BCH) + .put(C, CCH) + .put(D, DCH) + .put(E, ECH) + .put(F, FCH) + .put(G, GCH) + .build(); + + @Test + public void testNoneRoundTrip() + throws Exception + { + TupleDomain tupleDomain = TupleDomain.none(); + ExtractionResult result = fromPredicate(toPredicate(tupleDomain, COLUMN_HANDLES.inverse()), TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), tupleDomain); + } + + @Test + public void testAllRoundTrip() + throws Exception + { + TupleDomain tupleDomain = TupleDomain.all(); + ExtractionResult result = fromPredicate(toPredicate(tupleDomain, COLUMN_HANDLES.inverse()), TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), tupleDomain); + } + + @Test + public void testRoundTrip() + throws Exception + { + TupleDomain tupleDomain = withColumnDomains(ImmutableMap.builder() + .put(ACH, Domain.singleValue(1L)) + .put(BCH, Domain.onlyNull(Double.class)) + .put(CCH, Domain.notNull(String.class)) + .put(DCH, Domain.singleValue(true)) + .put(ECH, Domain.singleValue(2L)) + .put(FCH, Domain.create(SortedRangeSet.of(Range.lessThanOrEqual(1.1), Range.equal(2.0), Range.range(3.0, false, 3.5, true)), true)) + .put(GCH, Domain.create(SortedRangeSet.of(Range.lessThanOrEqual("2013-01-01"), Range.greaterThan("2013-10-01")), false)) + .build()); + + ExtractionResult result = fromPredicate(toPredicate(tupleDomain, COLUMN_HANDLES.inverse()), TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), tupleDomain); + } + + @Test + public void testToPredicateNone() + throws Exception + { + TupleDomain tupleDomain = withColumnDomains(ImmutableMap.builder() + .put(ACH, Domain.singleValue(1L)) + .put(BCH, Domain.onlyNull(Double.class)) + .put(CCH, Domain.notNull(String.class)) + .put(DCH, Domain.none(Boolean.class)) + .build()); + + Assert.assertEquals(toPredicate(tupleDomain, COLUMN_HANDLES.inverse()), FALSE_LITERAL); + } + + @Test + public void testToPredicateAllIgnored() + throws Exception + { + TupleDomain tupleDomain = withColumnDomains(ImmutableMap.builder() + .put(ACH, Domain.singleValue(1L)) + .put(BCH, Domain.onlyNull(Double.class)) + .put(CCH, Domain.notNull(String.class)) + .put(DCH, Domain.all(Boolean.class)) + .build()); + + ExtractionResult result = fromPredicate(toPredicate(tupleDomain, COLUMN_HANDLES.inverse()), TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.builder() + .put(ACH, Domain.singleValue(1L)) + .put(BCH, Domain.onlyNull(Double.class)) + .put(CCH, Domain.notNull(String.class)) + .build())); + } + + @Test + public void testToPredicate() + throws Exception + { + TupleDomain tupleDomain; + + tupleDomain = withColumnDomains(ImmutableMap.of(ACH, Domain.notNull(Long.class))); + Assert.assertEquals(toPredicate(tupleDomain, COLUMN_HANDLES.inverse()), isNotNull(A)); + + tupleDomain = withColumnDomains(ImmutableMap.of(ACH, Domain.onlyNull(Long.class))); + Assert.assertEquals(toPredicate(tupleDomain, COLUMN_HANDLES.inverse()), isNull(A)); + + tupleDomain = withColumnDomains(ImmutableMap.of(ACH, Domain.none(Long.class))); + Assert.assertEquals(toPredicate(tupleDomain, COLUMN_HANDLES.inverse()), FALSE_LITERAL); + + tupleDomain = withColumnDomains(ImmutableMap.of(ACH, Domain.all(Long.class))); + Assert.assertEquals(toPredicate(tupleDomain, COLUMN_HANDLES.inverse()), TRUE_LITERAL); + + tupleDomain = withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.greaterThan(1L)), false))); + Assert.assertEquals(toPredicate(tupleDomain, COLUMN_HANDLES.inverse()), greaterThan(A, longLiteral(1L))); + + tupleDomain = withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.greaterThanOrEqual(1L)), false))); + Assert.assertEquals(toPredicate(tupleDomain, COLUMN_HANDLES.inverse()), greaterThanOrEqual(A, longLiteral(1L))); + + tupleDomain = withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThan(1L)), false))); + Assert.assertEquals(toPredicate(tupleDomain, COLUMN_HANDLES.inverse()), lessThan(A, longLiteral(1L))); + + tupleDomain = withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.range(0L, false, 1L, true)), false))); + Assert.assertEquals(toPredicate(tupleDomain, COLUMN_HANDLES.inverse()), and(greaterThan(A, longLiteral(0L)), lessThanOrEqual(A, longLiteral(1L)))); + + tupleDomain = withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThanOrEqual(1L)), false))); + Assert.assertEquals(toPredicate(tupleDomain, COLUMN_HANDLES.inverse()), lessThanOrEqual(A, longLiteral(1L))); + + tupleDomain = withColumnDomains(ImmutableMap.of(ACH, Domain.singleValue(1L))); + Assert.assertEquals(toPredicate(tupleDomain, COLUMN_HANDLES.inverse()), equal(A, longLiteral(1L))); + + tupleDomain = withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.equal(1L), Range.equal(2L)), false))); + Assert.assertEquals(toPredicate(tupleDomain, COLUMN_HANDLES.inverse()), in(A, ImmutableList.of(1L, 2L))); + + tupleDomain = withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThan(1L)), true))); + Assert.assertEquals(toPredicate(tupleDomain, COLUMN_HANDLES.inverse()), or(lessThan(A, longLiteral(1L)), isNull(A))); + } + + @Test + public void testFromUnknownPredicate() + throws Exception + { + ExtractionResult result = fromPredicate(unprocessableExpression1(A), TYPES, COLUMN_HANDLES); + Assert.assertTrue(result.getTupleDomain().isAll()); + Assert.assertEquals(result.getRemainingExpression(), unprocessableExpression1(A)); + + // Test the complement + result = fromPredicate(not(unprocessableExpression1(A)), TYPES, COLUMN_HANDLES); + Assert.assertTrue(result.getTupleDomain().isAll()); + Assert.assertEquals(result.getRemainingExpression(), not(unprocessableExpression1(A))); + } + + @Test + public void testFromAndPredicate() + throws Exception + { + Expression originalPredicate = and( + and(greaterThan(A, longLiteral(1L)), unprocessableExpression1(A)), + and(lessThan(A, longLiteral(5L)), unprocessableExpression2(A))); + ExtractionResult result = fromPredicate(originalPredicate, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), and(unprocessableExpression1(A), unprocessableExpression2(A))); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.range(1L, false, 5L, false)), false)))); + + // Test complements + originalPredicate = not(and( + and(greaterThan(A, longLiteral(1L)), unprocessableExpression1(A)), + and(lessThan(A, longLiteral(5L)), unprocessableExpression2(A)))); + result = fromPredicate(originalPredicate, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), originalPredicate); + Assert.assertTrue(result.getTupleDomain().isAll()); + + originalPredicate = not(and( + not(and(greaterThan(A, longLiteral(1L)), unprocessableExpression1(A))), + not(and(lessThan(A, longLiteral(5L)), unprocessableExpression2(A))))); + result = fromPredicate(originalPredicate, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), originalPredicate); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.notNull(Long.class)))); + } + + @Test + public void testFromOrPredicate() + throws Exception + { + Expression originalPredicate = or( + and(greaterThan(A, longLiteral(1L)), unprocessableExpression1(A)), + and(lessThan(A, longLiteral(5L)), unprocessableExpression2(A))); + ExtractionResult result = fromPredicate(originalPredicate, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), originalPredicate); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.notNull(Long.class)))); + + originalPredicate = or( + and(equal(A, longLiteral(1L)), unprocessableExpression1(A)), + and(equal(A, longLiteral(2L)), unprocessableExpression2(A))); + result = fromPredicate(originalPredicate, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), originalPredicate); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.equal(1L), Range.equal(2L)), false)))); + + // Same unprocessableExpression means that we can do more extraction + // If both sides are operating on the same single symbol + originalPredicate = or( + and(equal(A, longLiteral(1L)), unprocessableExpression1(A)), + and(equal(A, longLiteral(2L)), unprocessableExpression1(A))); + result = fromPredicate(originalPredicate, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), unprocessableExpression1(A)); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.equal(1L), Range.equal(2L)), false)))); + + // And not if they have different symbols + originalPredicate = or( + and(equal(A, longLiteral(1L)), unprocessableExpression1(A)), + and(equal(B, doubleLiteral(2.0)), unprocessableExpression1(A))); + result = fromPredicate(originalPredicate, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), originalPredicate); + Assert.assertTrue(result.getTupleDomain().isAll()); + + // We can make another optimization if one side is the super set of the other side + originalPredicate = or( + and(greaterThan(A, longLiteral(1L)), greaterThan(B, doubleLiteral(1.0)), unprocessableExpression1(A)), + and(greaterThan(A, longLiteral(2L)), greaterThan(B, doubleLiteral(2.0)), unprocessableExpression1(A))); + result = fromPredicate(originalPredicate, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), unprocessableExpression1(A)); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of( + ACH, Domain.create(SortedRangeSet.of(Range.greaterThan(1L)), false), + BCH, Domain.create(SortedRangeSet.of(Range.greaterThan(1.0)), false)))); + + // We can't make those inferences if the unprocessableExpressions are non-deterministic + originalPredicate = or( + and(equal(A, longLiteral(1L)), randPredicate(A)), + and(equal(A, longLiteral(2L)), randPredicate(A))); + result = fromPredicate(originalPredicate, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), originalPredicate); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.equal(1L), Range.equal(2L)), false)))); + + // Test complements + originalPredicate = not(or( + and(greaterThan(A, longLiteral(1L)), unprocessableExpression1(A)), + and(lessThan(A, longLiteral(5L)), unprocessableExpression2(A)))); + result = fromPredicate(originalPredicate, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), and( + not(and(greaterThan(A, longLiteral(1L)), unprocessableExpression1(A))), + not(and(lessThan(A, longLiteral(5L)), unprocessableExpression2(A))))); + Assert.assertTrue(result.getTupleDomain().isAll()); + + originalPredicate = not(or( + not(and(greaterThan(A, longLiteral(1L)), unprocessableExpression1(A))), + not(and(lessThan(A, longLiteral(5L)), unprocessableExpression2(A))))); + result = fromPredicate(originalPredicate, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), and(unprocessableExpression1(A), unprocessableExpression2(A))); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.range(1L, false, 5L, false)), false)))); + } + + @Test + public void testFromNotPredicate() + throws Exception + { + Expression originalPredicate = not(and(equal(A, longLiteral(1L)), unprocessableExpression1(A))); + ExtractionResult result = fromPredicate(originalPredicate, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), originalPredicate); + Assert.assertTrue(result.getTupleDomain().isAll()); + + originalPredicate = not(unprocessableExpression1(A)); + result = fromPredicate(originalPredicate, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), originalPredicate); + Assert.assertTrue(result.getTupleDomain().isAll()); + + originalPredicate = not(TRUE_LITERAL); + result = fromPredicate(originalPredicate, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertTrue(result.getTupleDomain().isNone()); + + originalPredicate = not(equal(A, longLiteral(1L))); + result = fromPredicate(originalPredicate, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThan(1L), Range.greaterThan(1L)), false)))); + } + + @Test + public void testFromUnprocessableComparison() + throws Exception + { + // If it is not a simple comparison, we should not try to process it + Expression predicate = comparison(GREATER_THAN, unprocessableExpression1(A), unprocessableExpression2(A)); + ExtractionResult result = fromPredicate(predicate, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), predicate); + Assert.assertTrue(result.getTupleDomain().isAll()); + + // Complement + predicate = not(comparison(GREATER_THAN, unprocessableExpression1(A), unprocessableExpression2(A))); + result = fromPredicate(predicate, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), predicate); + Assert.assertTrue(result.getTupleDomain().isAll()); + } + + @Test + public void testFromBasicComparisons() + throws Exception + { + // Test out the extraction of all basic comparisons + Expression originalExpression = greaterThan(A, longLiteral(2L)); + ExtractionResult result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.greaterThan(2L)), false)))); + + originalExpression = greaterThanOrEqual(A, longLiteral(2L)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.greaterThanOrEqual(2L)), false)))); + + originalExpression = lessThan(A, longLiteral(2L)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThan(2L)), false)))); + + originalExpression = lessThanOrEqual(A, longLiteral(2L)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThanOrEqual(2L)), false)))); + + originalExpression = equal(A, longLiteral(2L)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.equal(2L)), false)))); + + originalExpression = notEqual(A, longLiteral(2L)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThan(2L), Range.greaterThan(2L)), false)))); + + originalExpression = isDistinctFrom(A, longLiteral(2L)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThan(2L), Range.greaterThan(2L)), true)))); + + // Test complement + originalExpression = not(greaterThan(A, longLiteral(2L))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThanOrEqual(2L)), false)))); + + originalExpression = not(greaterThanOrEqual(A, longLiteral(2L))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThan(2L)), false)))); + + originalExpression = not(lessThan(A, longLiteral(2L))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.greaterThanOrEqual(2L)), false)))); + + originalExpression = not(lessThanOrEqual(A, longLiteral(2L))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.greaterThan(2L)), false)))); + + originalExpression = not(equal(A, longLiteral(2L))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThan(2L), Range.greaterThan(2L)), false)))); + + originalExpression = not(notEqual(A, longLiteral(2L))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.equal(2L)), false)))); + + originalExpression = not(isDistinctFrom(A, longLiteral(2L))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.equal(2L)), false)))); + } + + @Test + public void testFromFlippedBasicComparisons() + throws Exception + { + // Test out the extraction of all basic comparisons where the reference literal ordering is flipped + ComparisonExpression originalExpression = comparison(GREATER_THAN, longLiteral(2L), reference(A)); + ExtractionResult result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThan(2L)), false)))); + + originalExpression = comparison(GREATER_THAN_OR_EQUAL, longLiteral(2L), reference(A)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThanOrEqual(2L)), false)))); + + originalExpression = comparison(LESS_THAN, longLiteral(2L), reference(A)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.greaterThan(2L)), false)))); + + originalExpression = comparison(LESS_THAN_OR_EQUAL, longLiteral(2L), reference(A)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.greaterThanOrEqual(2L)), false)))); + + originalExpression = comparison(EQUAL, longLiteral(2L), reference(A)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.equal(2L)), false)))); + + originalExpression = comparison(NOT_EQUAL, longLiteral(2L), reference(A)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThan(2L), Range.greaterThan(2L)), false)))); + + originalExpression = comparison(IS_DISTINCT_FROM, longLiteral(2L), reference(A)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThan(2L), Range.greaterThan(2L)), true)))); + + originalExpression = comparison(IS_DISTINCT_FROM, nullLiteral(), reference(A)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.notNull(Long.class)))); + } + + @Test + public void testFromBasicComparisonsWithNulls() + throws Exception + { + // Test out the extraction of all basic comparisons with null literals + Expression originalExpression = greaterThan(A, nullLiteral()); + ExtractionResult result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertTrue(result.getTupleDomain().isNone()); + + originalExpression = greaterThanOrEqual(A, nullLiteral()); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertTrue(result.getTupleDomain().isNone()); + + originalExpression = lessThan(A, nullLiteral()); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertTrue(result.getTupleDomain().isNone()); + + originalExpression = lessThanOrEqual(A, nullLiteral()); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertTrue(result.getTupleDomain().isNone()); + + originalExpression = equal(A, nullLiteral()); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertTrue(result.getTupleDomain().isNone()); + + originalExpression = notEqual(A, nullLiteral()); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertTrue(result.getTupleDomain().isNone()); + + originalExpression = isDistinctFrom(A, nullLiteral()); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.notNull(Long.class)))); + + // Test complements + originalExpression = not(greaterThan(A, nullLiteral())); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertTrue(result.getTupleDomain().isNone()); + + originalExpression = not(greaterThanOrEqual(A, nullLiteral())); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertTrue(result.getTupleDomain().isNone()); + + originalExpression = not(lessThan(A, nullLiteral())); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertTrue(result.getTupleDomain().isNone()); + + originalExpression = not(lessThanOrEqual(A, nullLiteral())); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertTrue(result.getTupleDomain().isNone()); + + originalExpression = not(equal(A, nullLiteral())); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertTrue(result.getTupleDomain().isNone()); + + originalExpression = not(notEqual(A, nullLiteral())); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertTrue(result.getTupleDomain().isNone()); + + originalExpression = not(isDistinctFrom(A, nullLiteral())); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.onlyNull(Long.class)))); + } + + @Test + public void testFromComparisonsWithImplictCoercions() + throws Exception + { + // B is a double column. Check that it can be compared against longs + Expression originalExpression = greaterThan(B, longLiteral(2L)); + ExtractionResult result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(BCH, Domain.create(SortedRangeSet.of(Range.greaterThan(2.0)), false)))); + + // C is a string column. Check that it can be compared. + originalExpression = greaterThan(C, stringLiteral("test")); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(CCH, Domain.create(SortedRangeSet.of(Range.greaterThan("test")), false)))); + + // A is a long column. Check that it can be compared against doubles + originalExpression = greaterThan(A, doubleLiteral(2.0)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.greaterThan(2L)), false)))); + + originalExpression = greaterThan(A, doubleLiteral(2.1)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.greaterThan(2L)), false)))); + + originalExpression = greaterThanOrEqual(A, doubleLiteral(2.0)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.greaterThanOrEqual(2L)), false)))); + + originalExpression = greaterThanOrEqual(A, doubleLiteral(2.1)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.greaterThanOrEqual(3L)), false)))); + + originalExpression = lessThan(A, doubleLiteral(2.0)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThan(2L)), false)))); + + originalExpression = lessThan(A, doubleLiteral(2.1)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThan(3L)), false)))); + + originalExpression = lessThanOrEqual(A, doubleLiteral(2.0)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThanOrEqual(2L)), false)))); + + originalExpression = lessThanOrEqual(A, doubleLiteral(2.1)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThanOrEqual(2L)), false)))); + + originalExpression = equal(A, doubleLiteral(2.0)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.equal(2L)), false)))); + + originalExpression = equal(A, doubleLiteral(2.1)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.none(Long.class)))); + + originalExpression = notEqual(A, doubleLiteral(2.0)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThan(2L), Range.greaterThan(2L)), false)))); + + originalExpression = notEqual(A, doubleLiteral(2.1)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.notNull(Long.class)))); + + originalExpression = isDistinctFrom(A, doubleLiteral(2.0)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThan(2L), Range.greaterThan(2L)), true)))); + + originalExpression = isDistinctFrom(A, doubleLiteral(2.1)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertTrue(result.getTupleDomain().isAll()); + + // Test complements + + // B is a double column. Check that it can be compared against longs + originalExpression = not(greaterThan(B, longLiteral(2L))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(BCH, Domain.create(SortedRangeSet.of(Range.lessThanOrEqual(2.0)), false)))); + + // C is a string column. Check that it can be compared. + originalExpression = not(greaterThan(C, stringLiteral("test"))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(CCH, Domain.create(SortedRangeSet.of(Range.lessThanOrEqual("test")), false)))); + + // A is a long column. Check that it can be compared against doubles + originalExpression = not(greaterThan(A, doubleLiteral(2.0))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThanOrEqual(2L)), false)))); + + originalExpression = not(greaterThan(A, doubleLiteral(2.1))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThanOrEqual(2L)), false)))); + + originalExpression = not(greaterThanOrEqual(A, doubleLiteral(2.0))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThan(2L)), false)))); + + originalExpression = not(greaterThanOrEqual(A, doubleLiteral(2.1))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThan(3L)), false)))); + + originalExpression = not(lessThan(A, doubleLiteral(2.0))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.greaterThanOrEqual(2L)), false)))); + + originalExpression = not(lessThan(A, doubleLiteral(2.1))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.greaterThanOrEqual(3L)), false)))); + + originalExpression = not(lessThanOrEqual(A, doubleLiteral(2.0))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.greaterThan(2L)), false)))); + + originalExpression = not(lessThanOrEqual(A, doubleLiteral(2.1))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.greaterThan(2L)), false)))); + + originalExpression = not(equal(A, doubleLiteral(2.0))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThan(2L), Range.greaterThan(2L)), false)))); + + originalExpression = not(equal(A, doubleLiteral(2.1))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.notNull(Long.class)))); + + originalExpression = not(notEqual(A, doubleLiteral(2.0))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.equal(2L)), false)))); + + originalExpression = not(notEqual(A, doubleLiteral(2.1))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.none(Long.class)))); + + originalExpression = not(isDistinctFrom(A, doubleLiteral(2.0))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.equal(2L)), false)))); + + originalExpression = not(isDistinctFrom(A, doubleLiteral(2.1))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertTrue(result.getTupleDomain().isNone()); + } + + @Test + public void testFromUnprocessableInPredicate() + throws Exception + { + Expression originalExpression = new InPredicate(unprocessableExpression1(A), new InListExpression(ImmutableList.of(TRUE_LITERAL))); + ExtractionResult result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), originalExpression); + Assert.assertTrue(result.getTupleDomain().isAll()); + + originalExpression = new InPredicate(reference(D), new InListExpression(ImmutableList.of(unprocessableExpression1(D)))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), equal(D, unprocessableExpression1(D))); + Assert.assertTrue(result.getTupleDomain().isAll()); + + originalExpression = new InPredicate(reference(D), new InListExpression(ImmutableList.of(TRUE_LITERAL, unprocessableExpression1(D)))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), or(equal(D, TRUE_LITERAL), equal(D, unprocessableExpression1(D)))); + Assert.assertTrue(result.getTupleDomain().isAll()); + + // Test complement + originalExpression = not(new InPredicate(reference(D), new InListExpression(ImmutableList.of(unprocessableExpression1(D))))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), not(equal(D, unprocessableExpression1(D)))); + Assert.assertTrue(result.getTupleDomain().isAll()); + } + + @Test + public void testFromInPredicate() + throws Exception + { + Expression originalExpression = in(A, ImmutableList.of(1L)); + ExtractionResult result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.singleValue(1L)))); + + originalExpression = in(A, ImmutableList.of(1L, 2L)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.equal(1L), Range.equal(2L)), false)))); + + originalExpression = not(in(A, ImmutableList.of(1L, 2L))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThan(1L), Range.range(1L, false, 2L, false), Range.greaterThan(2L)), false)))); + + originalExpression = in(A, Arrays.asList(1L, 2L, (Expression) null)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.equal(1L), Range.equal(2L)), false)))); + + originalExpression = not(in(A, Arrays.asList(1L, 2L, (Expression) null))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertTrue(result.getTupleDomain().isNone()); + + originalExpression = in(A, Arrays.asList((Expression) null)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertTrue(result.getTupleDomain().isNone()); + + originalExpression = not(in(A, Arrays.asList((Expression) null))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertTrue(result.getTupleDomain().isNone()); + } + + @Test + public void testFromBetweenPredicate() + throws Exception + { + Expression originalExpression = between(A, longLiteral(1L), longLiteral(2L)); + ExtractionResult result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.range(1L, true, 2L, true)), false)))); + + originalExpression = between(A, longLiteral(1L), doubleLiteral(2.1)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.range(1L, true, 2L, true)), false)))); + + originalExpression = between(A, longLiteral(1L), nullLiteral()); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertTrue(result.getTupleDomain().isNone()); + + // Test complements + originalExpression = not(between(A, longLiteral(1L), longLiteral(2L))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThan(1L), Range.greaterThan(2L)), false)))); + + originalExpression = not(between(A, longLiteral(1L), doubleLiteral(2.1))); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThan(1L), Range.greaterThan(2L)), false)))); + + originalExpression = not(between(A, longLiteral(1L), nullLiteral())); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.create(SortedRangeSet.of(Range.lessThan(1L)), false)))); + } + + @Test + public void testFromIsNullPredicate() + throws Exception + { + Expression originalExpression = isNull(A); + ExtractionResult result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.onlyNull(Long.class)))); + + originalExpression = not(isNull(A)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.notNull(Long.class)))); + } + + @Test + public void testFromIsNotNullPredicate() + throws Exception + { + Expression originalExpression = isNotNull(A); + ExtractionResult result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.notNull(Long.class)))); + + originalExpression = not(isNotNull(A)); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(ACH, Domain.onlyNull(Long.class)))); + } + + @Test + public void testFromBooleanLiteralPredicate() + throws Exception + { + Expression originalExpression = TRUE_LITERAL; + ExtractionResult result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertTrue(result.getTupleDomain().isAll()); + + originalExpression = not(TRUE_LITERAL); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertTrue(result.getTupleDomain().isNone()); + + originalExpression = FALSE_LITERAL; + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertTrue(result.getTupleDomain().isNone()); + + originalExpression = not(FALSE_LITERAL); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertTrue(result.getTupleDomain().isAll()); + } + + @Test + public void testFromNullLiteralPredicate() + throws Exception + { + Expression originalExpression = nullLiteral(); + ExtractionResult result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertTrue(result.getTupleDomain().isNone()); + + originalExpression = not(nullLiteral()); + result = fromPredicate(originalExpression, TYPES, COLUMN_HANDLES); + Assert.assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + Assert.assertTrue(result.getTupleDomain().isNone()); + } + + private static Expression unprocessableExpression1(Symbol symbol) + { + return comparison(GREATER_THAN, reference(symbol), reference(symbol)); + } + + private static Expression unprocessableExpression2(Symbol symbol) + { + return comparison(LESS_THAN, reference(symbol), reference(symbol)); + } + + private static Expression randPredicate(Symbol symbol) + { + return comparison(GREATER_THAN, reference(symbol), new FunctionCall(new QualifiedName("rand"), ImmutableList.of())); + } + + private static QualifiedNameReference reference(Symbol symbol) + { + return new QualifiedNameReference(symbol.toQualifiedName()); + } + + private static NotExpression not(Expression expression) + { + return new NotExpression(expression); + } + + private static ComparisonExpression comparison(ComparisonExpression.Type type, Expression expression1, Expression expression2) + { + return new ComparisonExpression(type, expression1, expression2); + } + + private static ComparisonExpression equal(Symbol symbol, Expression expression) + { + return comparison(EQUAL, reference(symbol), expression); + } + + private static ComparisonExpression notEqual(Symbol symbol, Expression expression) + { + return comparison(NOT_EQUAL, reference(symbol), expression); + } + + private static ComparisonExpression greaterThan(Symbol symbol, Expression expression) + { + return comparison(GREATER_THAN, reference(symbol), expression); + } + + private static ComparisonExpression greaterThanOrEqual(Symbol symbol, Expression expression) + { + return comparison(GREATER_THAN_OR_EQUAL, reference(symbol), expression); + } + + private static ComparisonExpression lessThan(Symbol symbol, Expression expression) + { + return comparison(LESS_THAN, reference(symbol), expression); + } + + private static ComparisonExpression lessThanOrEqual(Symbol symbol, Expression expression) + { + return comparison(LESS_THAN_OR_EQUAL, reference(symbol), expression); + } + + private static ComparisonExpression isDistinctFrom(Symbol symbol, Expression expression) + { + return comparison(IS_DISTINCT_FROM, reference(symbol), expression); + } + + private static IsNotNullPredicate isNotNull(Symbol symbol) + { + return new IsNotNullPredicate(reference(symbol)); + } + + private static IsNullPredicate isNull(Symbol symbol) + { + return new IsNullPredicate(reference(symbol)); + } + + private static InPredicate in(Symbol symbol, List values) + { + return new InPredicate(reference(symbol), new InListExpression(LiteralInterpreter.toExpressions(values))); + } + + private static BetweenPredicate between(Symbol symbol, Expression min, Expression max) + { + return new BetweenPredicate(reference(symbol), min, max); + } + + private static LongLiteral longLiteral(long value) + { + return new LongLiteral(Long.toString(value)); + } + + private static DoubleLiteral doubleLiteral(double value) + { + return new DoubleLiteral(Double.toString(value)); + } + + private static StringLiteral stringLiteral(String value) + { + return new StringLiteral(value); + } + + private static NullLiteral nullLiteral() + { + return new NullLiteral(); + } +} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestEffectivePredicateExtractor.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestEffectivePredicateExtractor.java index 58f7533eae8be..43ce68c1007e3 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestEffectivePredicateExtractor.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestEffectivePredicateExtractor.java @@ -17,6 +17,9 @@ import com.facebook.presto.connector.dual.DualTableHandle; import com.facebook.presto.metadata.FunctionHandle; import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.Domain; +import com.facebook.presto.spi.Partition; +import com.facebook.presto.spi.TupleDomain; import com.facebook.presto.sql.planner.plan.AggregationNode; import com.facebook.presto.sql.planner.plan.FilterNode; import com.facebook.presto.sql.planner.plan.JoinNode; @@ -40,12 +43,14 @@ import com.facebook.presto.sql.tree.QualifiedName; import com.facebook.presto.sql.tree.QualifiedNameReference; import com.facebook.presto.sql.tree.SortItem; +import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.base.Predicates; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableListMultimap; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; +import com.google.common.collect.Maps; import org.testng.Assert; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -59,9 +64,11 @@ import java.util.Set; import java.util.UUID; +import static com.facebook.presto.connector.dual.DualSplitManager.DualPartition; import static com.facebook.presto.sql.ExpressionUtils.and; import static com.facebook.presto.sql.ExpressionUtils.combineConjuncts; import static com.facebook.presto.sql.ExpressionUtils.or; +import static com.facebook.presto.sql.planner.plan.TableScanNode.GeneratedPartitions; public class TestEffectivePredicateExtractor { @@ -95,13 +102,13 @@ public void setUp() .put(F, new DualColumnHandle("f")) .build(); + Map assignments = Maps.filterKeys(scanAssignments, Predicates.in(ImmutableList.of(A, B, C, D, E, F))); baseTableScan = new TableScanNode( newId(), new DualTableHandle("default"), - ImmutableList.of(A, B, C, D, E, F), - scanAssignments, - BooleanLiteral.TRUE_LITERAL, - BooleanLiteral.TRUE_LITERAL + ImmutableList.copyOf(assignments.keySet()), + assignments, + Optional.absent() ); expressionNormalizer = new ExpressionIdentityNormalizer(); @@ -270,21 +277,126 @@ public void testWindow() public void testTableScan() throws Exception { + // Effective predicate is True if there are no generated partitions + Map assignments = Maps.filterKeys(scanAssignments, Predicates.in(ImmutableList.of(A, B, C, D))); PlanNode node = new TableScanNode( newId(), new DualTableHandle("default"), - ImmutableList.of(A, B, C), - scanAssignments, - and(greaterThan(BE, number(0)), equals(AE, DE), lessThan(DE, number(10)), lessThan(EE, number(3))), - BooleanLiteral.TRUE_LITERAL); - + ImmutableList.copyOf(assignments.keySet()), + assignments, + Optional.absent()); Expression effectivePredicate = EffectivePredicateExtractor.extract(node); + Assert.assertEquals(effectivePredicate, BooleanLiteral.TRUE_LITERAL); - // Only those that can be written in terms of the output symbols are extracted - Assert.assertEquals(normalizeConjuncts(effectivePredicate), - normalizeConjuncts( - greaterThan(BE, number(0)), - lessThan(AE, number(10)))); + // tupleDomainInput with no matching partitions + node = new TableScanNode( + newId(), + new DualTableHandle("default"), + ImmutableList.copyOf(assignments.keySet()), + assignments, + Optional.of(new GeneratedPartitions( + TupleDomain.withColumnDomains(ImmutableMap.of(scanAssignments.get(A), Domain.singleValue(1L))), + ImmutableList.of()))); + effectivePredicate = EffectivePredicateExtractor.extract(node); + Assert.assertEquals(effectivePredicate, BooleanLiteral.FALSE_LITERAL); + + // tupleDomainInput with non-descriptive partitions + node = new TableScanNode( + newId(), + new DualTableHandle("default"), + ImmutableList.copyOf(assignments.keySet()), + assignments, + Optional.of(new GeneratedPartitions( + TupleDomain.withColumnDomains(ImmutableMap.of(scanAssignments.get(A), Domain.singleValue(1L))), + ImmutableList.of(new DualPartition())))); + effectivePredicate = EffectivePredicateExtractor.extract(node); + Assert.assertEquals(normalizeConjuncts(effectivePredicate), normalizeConjuncts(equals(number(1L), AE))); + + // tupleDomainInput with descriptive partitions + node = new TableScanNode( + newId(), + new DualTableHandle("default"), + ImmutableList.copyOf(assignments.keySet()), + assignments, + Optional.of(new GeneratedPartitions( + TupleDomain.withColumnDomains(ImmutableMap.of(scanAssignments.get(A), Domain.singleValue(1L))), + ImmutableList.of(tupleDomainPartition(TupleDomain.withColumnDomains(ImmutableMap.of( + scanAssignments.get(A), Domain.singleValue(1L), + scanAssignments.get(B), Domain.singleValue(2L)))))))); + effectivePredicate = EffectivePredicateExtractor.extract(node); + Assert.assertEquals(normalizeConjuncts(effectivePredicate), normalizeConjuncts(equals(number(2L), BE), equals(number(1L), AE))); + + // generic tupleDomainInput with no matching partitions + node = new TableScanNode( + newId(), + new DualTableHandle("default"), + ImmutableList.copyOf(assignments.keySet()), + assignments, + Optional.of(new GeneratedPartitions( + TupleDomain.all(), + ImmutableList.of()))); + effectivePredicate = EffectivePredicateExtractor.extract(node); + Assert.assertEquals(effectivePredicate, BooleanLiteral.FALSE_LITERAL); + + // generic tupleDomainInput with non-descriptive partitions + node = new TableScanNode( + newId(), + new DualTableHandle("default"), + ImmutableList.copyOf(assignments.keySet()), + assignments, + Optional.of(new GeneratedPartitions( + TupleDomain.all(), + ImmutableList.of(new DualPartition())))); + effectivePredicate = EffectivePredicateExtractor.extract(node); + Assert.assertEquals(effectivePredicate, BooleanLiteral.TRUE_LITERAL); + + // generic tupleDomainInput with descriptive partitions + node = new TableScanNode( + newId(), + new DualTableHandle("default"), + ImmutableList.copyOf(assignments.keySet()), + assignments, + Optional.of(new GeneratedPartitions( + TupleDomain.all(), + ImmutableList.of(tupleDomainPartition(TupleDomain.withColumnDomains(ImmutableMap.of( + scanAssignments.get(A), Domain.singleValue(1L), + scanAssignments.get(B), Domain.singleValue(2L)))))))); + effectivePredicate = EffectivePredicateExtractor.extract(node); + Assert.assertEquals(normalizeConjuncts(effectivePredicate), normalizeConjuncts(equals(number(2L), BE), equals(number(1L), AE))); + + // Make sure only output symbols are produced + node = new TableScanNode( + newId(), + new DualTableHandle("default"), + ImmutableList.of(A), + assignments, + Optional.of(new GeneratedPartitions( + TupleDomain.withColumnDomains(ImmutableMap.of( + scanAssignments.get(A), Domain.singleValue(1L), + scanAssignments.get(D), Domain.singleValue(3L))), + ImmutableList.of(tupleDomainPartition(TupleDomain.withColumnDomains(ImmutableMap.of( + scanAssignments.get(A), Domain.singleValue(1L), + scanAssignments.get(C), Domain.singleValue(2L)))))))); + effectivePredicate = EffectivePredicateExtractor.extract(node); + Assert.assertEquals(normalizeConjuncts(effectivePredicate), normalizeConjuncts(equals(number(1L), AE))); + } + + private static Partition tupleDomainPartition(final TupleDomain tupleDomain) + { + return new Partition() + { + @Override + public String getPartitionId() + { + throw new UnsupportedOperationException(); + } + + @Override + public TupleDomain getTupleDomain() + { + return tupleDomain; + } + }; } @Test @@ -316,22 +428,22 @@ public void testInnerJoin() criteriaBuilder.add(new JoinNode.EquiJoinClause(B, E)); List criteria = criteriaBuilder.build(); + Map leftAssignments = Maps.filterKeys(scanAssignments, Predicates.in(ImmutableList.of(A, B, C))); TableScanNode leftScan = new TableScanNode( newId(), new DualTableHandle("default"), - ImmutableList.of(A, B, C), - scanAssignments, - BooleanLiteral.TRUE_LITERAL, - BooleanLiteral.TRUE_LITERAL + ImmutableList.copyOf(leftAssignments.keySet()), + leftAssignments, + Optional.absent() ); + Map rightAssignments = Maps.filterKeys(scanAssignments, Predicates.in(ImmutableList.of(D, E, F))); TableScanNode rightScan = new TableScanNode( newId(), new DualTableHandle("default"), - ImmutableList.of(D, E, F), - scanAssignments, - BooleanLiteral.TRUE_LITERAL, - BooleanLiteral.TRUE_LITERAL + ImmutableList.copyOf(rightAssignments.keySet()), + rightAssignments, + Optional.absent() ); PlanNode node = new JoinNode(newId(), @@ -367,22 +479,22 @@ public void testLeftJoin() criteriaBuilder.add(new JoinNode.EquiJoinClause(B, E)); List criteria = criteriaBuilder.build(); + Map leftAssignments = Maps.filterKeys(scanAssignments, Predicates.in(ImmutableList.of(A, B, C))); TableScanNode leftScan = new TableScanNode( newId(), new DualTableHandle("default"), - ImmutableList.of(A, B, C), - scanAssignments, - BooleanLiteral.TRUE_LITERAL, - BooleanLiteral.TRUE_LITERAL + ImmutableList.copyOf(leftAssignments.keySet()), + leftAssignments, + Optional.absent() ); + Map rightAssignments = Maps.filterKeys(scanAssignments, Predicates.in(ImmutableList.of(D, E, F))); TableScanNode rightScan = new TableScanNode( newId(), new DualTableHandle("default"), - ImmutableList.of(D, E, F), - scanAssignments, - BooleanLiteral.TRUE_LITERAL, - BooleanLiteral.TRUE_LITERAL + ImmutableList.copyOf(rightAssignments.keySet()), + rightAssignments, + Optional.absent() ); PlanNode node = new JoinNode(newId(), @@ -418,22 +530,22 @@ public void testRightJoin() criteriaBuilder.add(new JoinNode.EquiJoinClause(B, E)); List criteria = criteriaBuilder.build(); + Map leftAssignments = Maps.filterKeys(scanAssignments, Predicates.in(ImmutableList.of(A, B, C))); TableScanNode leftScan = new TableScanNode( newId(), new DualTableHandle("default"), - ImmutableList.of(A, B, C), - scanAssignments, - BooleanLiteral.TRUE_LITERAL, - BooleanLiteral.TRUE_LITERAL + ImmutableList.copyOf(leftAssignments.keySet()), + leftAssignments, + Optional.absent() ); + Map rightAssignments = Maps.filterKeys(scanAssignments, Predicates.in(ImmutableList.of(D, E, F))); TableScanNode rightScan = new TableScanNode( newId(), new DualTableHandle("default"), - ImmutableList.of(D, E, F), - scanAssignments, - BooleanLiteral.TRUE_LITERAL, - BooleanLiteral.TRUE_LITERAL + ImmutableList.copyOf(rightAssignments.keySet()), + rightAssignments, + Optional.absent() ); PlanNode node = new JoinNode(newId(), diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestingColumnHandle.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestingColumnHandle.java new file mode 100644 index 0000000000000..112fb89725e0d --- /dev/null +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestingColumnHandle.java @@ -0,0 +1,62 @@ +/* + * 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.google.common.base.Objects; + +import static com.google.common.base.Preconditions.checkNotNull; + +public class TestingColumnHandle + implements ColumnHandle +{ + private final Symbol symbol; + + public TestingColumnHandle(Symbol symbol) + { + this.symbol = checkNotNull(symbol, "symbol is null"); + } + + public Symbol getSymbol() + { + return symbol; + } + + @Override + public int hashCode() + { + return Objects.hashCode(symbol); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final TestingColumnHandle other = (TestingColumnHandle) obj; + return Objects.equal(this.symbol, other.symbol); + } + + @Override + public String toString() + { + return Objects.toStringHelper(this) + .add("symbol", symbol) + .toString(); + } +} diff --git a/presto-main/src/test/java/com/facebook/presto/util/LocalQueryRunner.java b/presto-main/src/test/java/com/facebook/presto/util/LocalQueryRunner.java index 222911172667d..75b7f8a71eac5 100644 --- a/presto-main/src/test/java/com/facebook/presto/util/LocalQueryRunner.java +++ b/presto-main/src/test/java/com/facebook/presto/util/LocalQueryRunner.java @@ -45,8 +45,10 @@ import com.facebook.presto.operator.TaskContext; import com.facebook.presto.spi.ConnectorSplitManager; import com.facebook.presto.spi.Partition; +import com.facebook.presto.spi.PartitionResult; import com.facebook.presto.spi.Split; import com.facebook.presto.spi.SystemTable; +import com.facebook.presto.spi.TupleDomain; import com.facebook.presto.split.DataStreamManager; import com.facebook.presto.split.DataStreamProvider; import com.facebook.presto.split.SplitManager; @@ -76,7 +78,6 @@ import com.facebook.presto.tpch.TpchSplitManager; import com.facebook.presto.tuple.TupleInfo; import com.google.common.base.Optional; -import com.google.common.base.Predicates; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import io.airlift.node.NodeConfig; @@ -206,7 +207,7 @@ public List createDrivers(@Language("SQL") String sql, OutputFactory out } PlanNodeIdAllocator idAllocator = new PlanNodeIdAllocator(); - PlanOptimizersFactory planOptimizersFactory = new PlanOptimizersFactory(metadata); + PlanOptimizersFactory planOptimizersFactory = new PlanOptimizersFactory(metadata, splitManager); QueryExplainer queryExplainer = new QueryExplainer(session, planOptimizersFactory.get(), metadata, new MockPeriodicImportManager(), new MockStorageManager()); Analyzer analyzer = new Analyzer(session, metadata, Optional.of(queryExplainer)); @@ -243,12 +244,7 @@ public List createDrivers(@Language("SQL") String sql, OutputFactory out for (PlanNode sourceNode : subplan.getFragment().getSources()) { TableScanNode tableScan = (TableScanNode) sourceNode; - DataSource dataSource = splitManager.getSplits(session, - tableScan.getTable(), - tableScan.getPartitionPredicate(), - tableScan.getUpstreamPredicateHint(), - Predicates.alwaysTrue(), - tableScan.getAssignments()); + DataSource dataSource = splitManager.getPartitionSplits(tableScan.getTable(), getPartitions(tableScan)); ImmutableSet.Builder scheduledSplits = ImmutableSet.builder(); for (Split split : dataSource.getSplits()) { @@ -281,6 +277,17 @@ public List createDrivers(@Language("SQL") String sql, OutputFactory out return ImmutableList.copyOf(drivers); } + private List getPartitions(TableScanNode node) + { + if (node.getGeneratedPartitions().isPresent()) { + return node.getGeneratedPartitions().get().getPartitions(); + } + + // Otherwise return all partitions + PartitionResult matchingPartitions = splitManager.getPartitions(node.getTable(), Optional.absent()); + return matchingPartitions.getPartitions(); + } + public static LocalQueryRunner createDualLocalQueryRunner(ExecutorService executor) { return createDualLocalQueryRunner(new Session("user", "test", DEFAULT_CATALOG, DEFAULT_SCHEMA, null, null), executor); @@ -291,7 +298,7 @@ public static LocalQueryRunner createDualLocalQueryRunner(Session session, Execu InMemoryNodeManager nodeManager = new InMemoryNodeManager(); MetadataManager metadataManager = new MetadataManager(); - SplitManager splitManager = new SplitManager(metadataManager, ImmutableSet.of()); + SplitManager splitManager = new SplitManager(ImmutableSet.of()); DataStreamManager dataStreamManager = new DataStreamManager(); addDual(nodeManager, metadataManager, splitManager, dataStreamManager); @@ -320,7 +327,7 @@ public static LocalQueryRunner createTpchLocalQueryRunner(Session session, TpchB InMemoryNodeManager nodeManager = new InMemoryNodeManager(); MetadataManager metadataManager = new MetadataManager(); - SplitManager splitManager = new SplitManager(metadataManager, ImmutableSet.of()); + SplitManager splitManager = new SplitManager(ImmutableSet.of()); DataStreamManager dataStreamManager = new DataStreamManager(); addDual(nodeManager, metadataManager, splitManager, dataStreamManager); diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index d10736355aeec..3384f9a97a343 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -21,5 +21,27 @@ com.fasterxml.jackson.core jackson-annotations + + + com.fasterxml.jackson.core + jackson-databind + + + + com.fasterxml.jackson.core + jackson-core + + + + org.testng + testng + test + + + + com.google.guava + guava + test + diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/ColumnType.java b/presto-spi/src/main/java/com/facebook/presto/spi/ColumnType.java index 006390499cb9c..4ccf72b4e69fa 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/ColumnType.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/ColumnType.java @@ -13,7 +13,34 @@ */ package com.facebook.presto.spi; +import java.util.Objects; + public enum ColumnType { - BOOLEAN, LONG, DOUBLE, STRING + BOOLEAN(Boolean.class), + LONG(Long.class), + DOUBLE(Double.class), + STRING(String.class); + + private final Class nativeType; + + private ColumnType(Class nativeType) + { + this.nativeType = Objects.requireNonNull(nativeType, "nativeType is null"); + } + + public Class getNativeType() + { + return nativeType; + } + + public static ColumnType fromNativeType(Class nativeType) + { + for (ColumnType columnType : ColumnType.values()) { + if (columnType.getNativeType().equals(nativeType)) { + return columnType; + } + } + throw new IllegalArgumentException(String.format("No native column type found for %s", nativeType)); + } } diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/ConnectorSplitManager.java b/presto-spi/src/main/java/com/facebook/presto/spi/ConnectorSplitManager.java index ff5ef08405adf..0cd924595ee42 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/ConnectorSplitManager.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/ConnectorSplitManager.java @@ -14,15 +14,30 @@ package com.facebook.presto.spi; import java.util.List; -import java.util.Map; public interface ConnectorSplitManager { + /** + * Get the globally-unique id of this connector instance + */ String getConnectorId(); + /** + * Returns true only if this ConnectorSplitManager can operate on the TableHandle + */ boolean canHandle(TableHandle handle); - List getPartitions(TableHandle table, Map bindings); + /** + * Gets the Partitions for the specified table. + * + * The TupleDomain indicates the execution filters that will be directly applied to the + * data stream produced by this connector. Connectors are encouraged to take advantage of + * this information to perform connector-specific optimizations. + */ + PartitionResult getPartitions(TableHandle table, TupleDomain tupleDomain); + /** + * Gets the Splits for the specified Partitions in the indicated table. + */ Iterable getPartitionSplits(TableHandle table, List partitions); } diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/Domain.java b/presto-spi/src/main/java/com/facebook/presto/spi/Domain.java new file mode 100644 index 0000000000000..0ae99f81269c3 --- /dev/null +++ b/presto-spi/src/main/java/com/facebook/presto/spi/Domain.java @@ -0,0 +1,207 @@ +/* + * 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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +/** + * Defines the possible values of a single variable in terms of its valid scalar ranges and nullability. + * + * For example: + * Domain.none() => no scalar values allowed, NULL not allowed + * Domain.all() => all scalar values allowed, NULL allowed + * Domain.onlyNull() => no scalar values allowed, NULL allowed + * Domain.notNull() => all scalar values allowed, NULL not allowed + */ +public final class Domain +{ + private final SortedRangeSet ranges; + private final boolean nullAllowed; + + @JsonCreator + public Domain( + @JsonProperty("ranges") SortedRangeSet ranges, + @JsonProperty("nullAllowed") boolean nullAllowed) + { + this.ranges = Objects.requireNonNull(ranges, "ranges is null"); + this.nullAllowed = nullAllowed; + } + + public static Domain create(SortedRangeSet ranges, boolean nullAllowed) + { + return new Domain(ranges, nullAllowed); + } + + public static Domain none(Class type) + { + return new Domain(SortedRangeSet.none(type), false); + } + + public static Domain all(Class type) + { + return new Domain(SortedRangeSet.of(Range.all(type)), true); + } + + public static Domain onlyNull(Class type) + { + return new Domain(SortedRangeSet.none(type), true); + } + + public static Domain notNull(Class type) + { + return new Domain(SortedRangeSet.all(type), false); + } + + public static Domain singleValue(Comparable value) + { + return new Domain(SortedRangeSet.of(Range.equal(value)), false); + } + + @JsonIgnore + public Class getType() + { + return ranges.getType(); + } + + /** + * Returns a SortedRangeSet to represent the set of scalar values that are allowed in this Domain. + * An empty (a.k.a. "none") SortedRangeSet indicates that no scalar values are allowed. + */ + @JsonProperty + public SortedRangeSet getRanges() + { + return ranges; + } + + @JsonProperty + public boolean isNullAllowed() + { + return nullAllowed; + } + + @JsonIgnore + public boolean isNone() + { + return equals(Domain.none(getType())); + } + + @JsonIgnore + public boolean isAll() + { + return equals(Domain.all(getType())); + } + + @JsonIgnore + public boolean isSingleValue() + { + return !nullAllowed && ranges.isSingleValue(); + } + + @JsonIgnore + public Comparable getSingleValue() + { + if (!isSingleValue()) { + throw new IllegalStateException("Domain is not a single value"); + } + return ranges.getSingleValue(); + } + + public boolean includesValue(Comparable value) + { + Objects.requireNonNull(value, "value is null"); + return ranges.includesMarker(Marker.exactly(value)); + } + + public boolean overlaps(Domain other) + { + checkTypeCompatibility(other); + return !this.intersect(other).isNone(); + } + + public boolean contains(Domain other) + { + checkTypeCompatibility(other); + return this.union(other).equals(this); + } + + public Domain intersect(Domain other) + { + checkTypeCompatibility(other); + SortedRangeSet intersectedRanges = this.getRanges().intersect(other.getRanges()); + boolean nullAllowed = this.isNullAllowed() && other.isNullAllowed(); + return new Domain(intersectedRanges, nullAllowed); + } + + public Domain union(Domain other) + { + checkTypeCompatibility(other); + SortedRangeSet unionRanges = this.getRanges().union(other.getRanges()); + boolean nullAllowed = this.isNullAllowed() || other.isNullAllowed(); + return new Domain(unionRanges, nullAllowed); + } + + public Domain complement() + { + return new Domain(ranges.complement(), !nullAllowed); + } + + public Domain subtract(Domain other) + { + checkTypeCompatibility(other); + SortedRangeSet subtractedRanges = this.getRanges().subtract(other.getRanges()); + boolean nullAllowed = this.isNullAllowed() && !other.isNullAllowed(); + return new Domain(subtractedRanges, nullAllowed); + } + + private void checkTypeCompatibility(Domain domain) + { + if (!getType().equals(domain.getType())) { + throw new IllegalArgumentException(String.format("Mismatched Domain types: %s vs %s", getType(), domain.getType())); + } + } + + @Override + public int hashCode() + { + return Objects.hash(ranges, nullAllowed); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final Domain other = (Domain) obj; + return Objects.equals(this.ranges, other.ranges) && + Objects.equals(this.nullAllowed, other.nullAllowed); + } + + @Override + public String toString() + { + final StringBuilder sb = new StringBuilder("Domain{"); + sb.append("ranges=").append(ranges); + sb.append(", nullAllowed=").append(nullAllowed); + sb.append('}'); + return sb.toString(); + } +} diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/Marker.java b/presto-spi/src/main/java/com/facebook/presto/spi/Marker.java new file mode 100644 index 0000000000000..15ada777ba318 --- /dev/null +++ b/presto-spi/src/main/java/com/facebook/presto/spi/Marker.java @@ -0,0 +1,296 @@ +/* + * 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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.lang.reflect.ParameterizedType; +import java.lang.reflect.Type; +import java.util.Objects; + +/** + * A point on the continuous space defined by the specified type. + * Each point may be just below, exact, or just above the specified value according to the Bound. + */ +public final class Marker + implements Comparable +{ + public static enum Bound + { + BELOW, // lower than the value, but infinitesimally close to the value + EXACTLY, // exactly the value + ABOVE // higher than the value, but infinitesimally close to the value + } + + private final Class type; + private final Comparable value; + private final Bound bound; + + /** + * LOWER UNBOUNDED is specified with a null value and a ABOVE bound + * UPPER UNBOUNDED is specified with a null value and a BELOW bound + */ + private Marker(Class type, Comparable value, Bound bound) + { + Objects.requireNonNull(type, "type is null"); + Objects.requireNonNull(bound, "bound is null"); + if (!verifySelfComparable(type)) { + throw new IllegalArgumentException("type must be comparable to itself: " + type); + } + if (value == null && bound == Bound.EXACTLY) { + throw new IllegalArgumentException("Can not be equal to unbounded"); + } + if (value != null && !type.isInstance(value)) { + throw new IllegalArgumentException(String.format("value (%s) must be of specified type (%s)", value, type)); + } + this.type = type; + this.value = value; + this.bound = bound; + } + + @JsonCreator + public Marker( + @JsonProperty("value") SerializableNativeValue value, + @JsonProperty("bound") Bound bound) + { + this(value.getType(), value.getValue(), bound); + } + + private static boolean verifySelfComparable(Class type) + { + // TODO: expand this with the proper implementation + for (Type interfaceType : type.getGenericInterfaces()) { + if (interfaceType instanceof ParameterizedType) { + ParameterizedType parameterizedType = (ParameterizedType) interfaceType; + if (parameterizedType.getRawType().equals(Comparable.class)) { + Type[] actualTypeArguments = parameterizedType.getActualTypeArguments(); + Type typeArgument = actualTypeArguments[0]; + if (typeArgument.equals(type)) { + return true; + } + } + } + } + return false; + } + + public static Marker upperUnbounded(Class type) + { + return new Marker(type, null, Bound.BELOW); + } + + public static Marker lowerUnbounded(Class type) + { + return new Marker(type, null, Bound.ABOVE); + } + + public static Marker above(Comparable value) + { + Objects.requireNonNull(value, "value is null"); + return new Marker(value.getClass(), value, Bound.ABOVE); + } + + public static Marker exactly(Comparable value) + { + Objects.requireNonNull(value, "value is null"); + return new Marker(value.getClass(), value, Bound.EXACTLY); + } + + public static Marker below(Comparable value) + { + Objects.requireNonNull(value, "value is null"); + return new Marker(value.getClass(), value, Bound.BELOW); + } + + @JsonIgnore + public Class getType() + { + return type; + } + + @JsonIgnore + public Comparable getValue() + { + if (value == null) { + throw new IllegalStateException("Can not get value for unbounded"); + } + return value; + } + + @JsonProperty("value") + public SerializableNativeValue getSerializableNativeValue() + { + return new SerializableNativeValue(type, value); + } + + @JsonProperty + public Bound getBound() + { + return bound; + } + + @JsonIgnore + public boolean isUpperUnbounded() + { + return value == null && bound == Bound.BELOW; + } + + @JsonIgnore + public boolean isLowerUnbounded() + { + return value == null && bound == Bound.ABOVE; + } + + private void checkTypeCompatibility(Marker marker) + { + if (!type.equals(marker.getType())) { + throw new IllegalArgumentException(String.format("Mismatched Marker types: %s vs %s", type, marker.getType())); + } + } + + /** + * Adjacency is defined by two Markers being infinitesimally close to each other. + * This means they must share the same value and have adjacent Bounds. + */ + public boolean isAdjacent(Marker other) + { + checkTypeCompatibility(other); + if (isUpperUnbounded() || isLowerUnbounded() || other.isUpperUnbounded() || other.isLowerUnbounded()) { + return false; + } + if (compare(value, other.value) != 0) { + return false; + } + return (bound == Bound.EXACTLY && other.bound != Bound.EXACTLY) || + (bound != Bound.EXACTLY && other.bound == Bound.EXACTLY); + } + + public Marker greaterAdjacent() + { + if (value == null) { + throw new IllegalStateException("No marker adjacent to unbounded"); + } + switch (bound) { + case BELOW: + return new Marker(type, value, Bound.EXACTLY); + case EXACTLY: + return new Marker(type, value, Bound.ABOVE); + case ABOVE: + throw new IllegalStateException("No greater marker adjacent to an ABOVE bound"); + default: + throw new AssertionError("Unsupported type: " + bound); + } + } + + public Marker lesserAdjacent() + { + if (value == null) { + throw new IllegalStateException("No marker adjacent to unbounded"); + } + switch (bound) { + case BELOW: + throw new IllegalStateException("No lesser marker adjacent to a BELOW bound"); + case EXACTLY: + return new Marker(type, value, Bound.BELOW); + case ABOVE: + return new Marker(type, value, Bound.EXACTLY); + default: + throw new AssertionError("Unsupported type: " + bound); + } + } + + @Override + public int compareTo(Marker o) + { + checkTypeCompatibility(o); + if (isUpperUnbounded()) { + return o.isUpperUnbounded() ? 0 : 1; + } + if (isLowerUnbounded()) { + return o.isLowerUnbounded() ? 0 : -1; + } + if (o.isUpperUnbounded()) { + return -1; + } + if (o.isLowerUnbounded()) { + return 1; + } + // INVARIANT: value and o.value not null + + int compare = compare(value, o.value); + if (compare == 0) { + if (bound == o.bound) { + return 0; + } + if (bound == Bound.BELOW) { + return -1; + } + if (bound == Bound.ABOVE) { + return 1; + } + // INVARIANT: bound == EXACTLY + return (o.bound == Bound.BELOW) ? 1 : -1; + } + return compare; + } + + @SuppressWarnings({ "unchecked", "rawtypes" }) + private static int compare(Comparable value1, Comparable value2) + { + // This is terrible, but it should be safe as we have checked the compatibility in the constructor + return ((Comparable) value1).compareTo(value2); + } + + public static Marker min(Marker marker1, Marker marker2) + { + return marker1.compareTo(marker2) <= 0 ? marker1 : marker2; + } + + public static Marker max(Marker marker1, Marker marker2) + { + return marker1.compareTo(marker2) >= 0 ? marker1 : marker2; + } + + @Override + public int hashCode() + { + return Objects.hash(type, value, bound); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final Marker other = (Marker) obj; + return Objects.equals(this.type, other.type) && Objects.equals(this.value, other.value) && Objects.equals(this.bound, other.bound); + } + + @Override + public String toString() + { + final StringBuilder sb = new StringBuilder("Marker{"); + sb.append("type=").append(type); + sb.append(", value=").append(value); + sb.append(", bound=").append(bound); + sb.append('}'); + return sb.toString(); + } +} diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/Partition.java b/presto-spi/src/main/java/com/facebook/presto/spi/Partition.java index 4ed06453cd6ad..98e9254a5d9e5 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/Partition.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/Partition.java @@ -13,17 +13,15 @@ */ package com.facebook.presto.spi; -import java.util.Map; - public interface Partition { /** - * Get the unique id if this partition within the scope of the table. + * Get the unique id of this partition within the scope of the table. */ String getPartitionId(); /** - * Gets the values associated with each partition key for this partition. + * Gets the TupleDomain that provides a bound on the tuples within this Partition. */ - Map getKeys(); + TupleDomain getTupleDomain(); } diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/PartitionResult.java b/presto-spi/src/main/java/com/facebook/presto/spi/PartitionResult.java new file mode 100644 index 0000000000000..b8c333108904f --- /dev/null +++ b/presto-spi/src/main/java/com/facebook/presto/spi/PartitionResult.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.spi; + +import java.util.List; + +import static java.util.Objects.requireNonNull; + +/** + * Result of fetching Partitions in the ConnectorSplitManager interface. + * + * Results are comprised of two parts: + * 1) The actual partitions + * 2) The TupleDomain that represents the values that the connector was not able to pre-evaluate + * when generating the partitions and will need to be double-checked by the final execution plan. + */ +public class PartitionResult +{ + private final List partitions; + private final TupleDomain undeterminedTupleDomain; + + public PartitionResult(List partitions, TupleDomain undeterminedTupleDomain) + { + this.partitions = requireNonNull(partitions, "partitions is null"); + this.undeterminedTupleDomain = requireNonNull(undeterminedTupleDomain, "undeterminedTupleDomain is null"); + } + + public List getPartitions() + { + return partitions; + } + + public TupleDomain getUndeterminedTupleDomain() + { + return undeterminedTupleDomain; + } +} diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/Range.java b/presto-spi/src/main/java/com/facebook/presto/spi/Range.java new file mode 100644 index 0000000000000..74ee548497369 --- /dev/null +++ b/presto-spi/src/main/java/com/facebook/presto/spi/Range.java @@ -0,0 +1,225 @@ +/* + * 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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Objects; + +/** + * A Range of values across the continuous space defined by the types of the Markers + */ +public final class Range +{ + private final Marker low; + private final Marker high; + + @JsonCreator + public Range( + @JsonProperty("low") Marker low, + @JsonProperty("high") Marker high) + { + Objects.requireNonNull(low, "value is null"); + Objects.requireNonNull(high, "value is null"); + if (!low.getType().equals(high.getType())) { + throw new IllegalArgumentException(String.format("Marker types do not match: %s vs %s", low.getType(), high.getType())); + } + if (low.isUpperUnbounded()) { + throw new IllegalArgumentException("low cannot be upper unbounded"); + } + if (high.isLowerUnbounded()) { + throw new IllegalArgumentException("high cannot be lower unbounded"); + } + if (low.compareTo(high) > 0) { + throw new IllegalArgumentException("low must be less than or equal to high"); + } + this.low = low; + this.high = high; + } + + public static Range all(Class type) + { + return new Range(Marker.lowerUnbounded(type), Marker.upperUnbounded(type)); + } + + public static Range greaterThan(Comparable low) + { + return new Range(Marker.above(low), Marker.upperUnbounded(low.getClass())); + } + + public static Range greaterThanOrEqual(Comparable low) + { + return new Range(Marker.exactly(low), Marker.upperUnbounded(low.getClass())); + } + + public static Range lessThan(Comparable high) + { + return new Range(Marker.lowerUnbounded(high.getClass()), Marker.below(high)); + } + + public static Range lessThanOrEqual(Comparable high) + { + return new Range(Marker.lowerUnbounded(high.getClass()), Marker.exactly(high)); + } + + public static Range equal(Comparable value) + { + return new Range(Marker.exactly(value), Marker.exactly(value)); + } + + public static Range range(Comparable low, boolean lowInclusive, Comparable high, boolean highInclusive) + { + Marker lowMarker = lowInclusive ? Marker.exactly(low) : Marker.above(low); + Marker highMarker = highInclusive ? Marker.exactly(high) : Marker.below(high); + return new Range(lowMarker, highMarker); + } + + @JsonIgnore + public Class getType() + { + return low.getType(); + } + + @JsonProperty + public Marker getLow() + { + return low; + } + + @JsonProperty + public Marker getHigh() + { + return high; + } + + @JsonIgnore + public boolean isSingleValue() + { + return !low.isLowerUnbounded() && + !high.isUpperUnbounded() && + low.getBound() == Marker.Bound.EXACTLY && + high.getBound() == Marker.Bound.EXACTLY && + low.getValue() == high.getValue(); + } + + @JsonIgnore + public Comparable getSingleValue() + { + if (!isSingleValue()) { + throw new IllegalStateException("Range does not have just a single value"); + } + return low.getValue(); + } + + @JsonIgnore + public boolean isAll() + { + return low.isLowerUnbounded() && high.isUpperUnbounded(); + } + + public boolean includes(Marker marker) + { + Objects.requireNonNull(marker, "marker is null"); + checkTypeCompatibility(marker); + return low.compareTo(marker) <= 0 && high.compareTo(marker) >= 0; + } + + public boolean contains(Range other) + { + checkTypeCompatibility(other); + return this.getLow().compareTo(other.getLow()) <= 0 && + this.getHigh().compareTo(other.getHigh()) >= 0; + } + + public Range span(Range other) + { + checkTypeCompatibility(other); + Marker lowMarker = Marker.min(low, other.getLow()); + Marker highMarker = Marker.max(high, other.getHigh()); + return new Range(lowMarker, highMarker); + } + + public boolean overlaps(Range other) + { + checkTypeCompatibility(other); + return this.getLow().compareTo(other.getHigh()) <= 0 && + other.getLow().compareTo(this.getHigh()) <= 0; + } + + public Range intersect(Range other) + { + checkTypeCompatibility(other); + if (!this.overlaps(other)) { + throw new IllegalArgumentException("Cannot intersect non-overlapping ranges"); + } + Marker lowMarker = Marker.max(low, other.getLow()); + Marker highMarker = Marker.min(high, other.getHigh()); + return new Range(lowMarker, highMarker); + } + + private void checkTypeCompatibility(Range range) + { + if (!getType().equals(range.getType())) { + throw new IllegalArgumentException(String.format("Mismatched Range types: %s vs %s", getType(), range.getType())); + } + } + + private void checkTypeCompatibility(Marker marker) + { + if (!getType().equals(marker.getType())) { + throw new IllegalArgumentException(String.format("Marker of %s does not match Range of %s", marker.getType(), getType())); + } + } + + @Override + public int hashCode() + { + return Objects.hash(low, high); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final Range other = (Range) obj; + return Objects.equals(this.low, other.low) && + Objects.equals(this.high, other.high); + } + + @Override + public String toString() + { + final StringBuilder sb = new StringBuilder(); + if (isSingleValue()) { + sb.append('[').append(low.getValue()).append(']'); + } + else { + sb.append((low.getBound() == Marker.Bound.EXACTLY) ? '[' : '('); + sb.append(low.isLowerUnbounded() ? "" : low.getValue()); + sb.append(", "); + sb.append(high.isUpperUnbounded() ? "" : high.getValue()); + sb.append((high.getBound() == Marker.Bound.EXACTLY) ? ']' : ')'); + } + return sb.toString(); + } +} diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/SerializableNativeValue.java b/presto-spi/src/main/java/com/facebook/presto/spi/SerializableNativeValue.java new file mode 100644 index 0000000000000..3fff61586cc38 --- /dev/null +++ b/presto-spi/src/main/java/com/facebook/presto/spi/SerializableNativeValue.java @@ -0,0 +1,157 @@ +/* + * 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; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonToken; +import com.fasterxml.jackson.core.io.SerializedString; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonDeserializer; +import com.fasterxml.jackson.databind.JsonSerializer; +import com.fasterxml.jackson.databind.SerializerProvider; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.fasterxml.jackson.databind.annotation.JsonSerialize; + +import java.io.IOException; +import java.util.Objects; + +@JsonSerialize(using = SerializableNativeValue.Serializer.class) +@JsonDeserialize(using = SerializableNativeValue.Deserializer.class) +public class SerializableNativeValue +{ + private final Class type; + private final Comparable value; + + public SerializableNativeValue(Class type, Comparable value) + { + this.type = Objects.requireNonNull(type, "type is null"); + this.value = value; + if (value != null && !type.isInstance(value)) { + throw new IllegalArgumentException(String.format("type %s does not match value %s", type.getClass(), value)); + } + } + + public Class getType() + { + return type; + } + + public Comparable getValue() + { + return value; + } + + public static class Serializer + extends JsonSerializer + { + @Override + public void serialize(SerializableNativeValue value, JsonGenerator generator, SerializerProvider provider) + throws IOException + { + generator.writeStartObject(); + generator.writeStringField("type", value.getType().getCanonicalName()); + generator.writeFieldName("value"); + if (value.getValue() == null) { + generator.writeNull(); + } + else { + writeValue(value, generator); + } + generator.writeEndObject(); + } + + private static void writeValue(SerializableNativeValue value, JsonGenerator jsonGenerator) + throws IOException + { + ColumnType columnType = ColumnType.fromNativeType(value.getType()); + switch (columnType) { + case STRING: + jsonGenerator.writeString((String) value.getValue()); + break; + case BOOLEAN: + jsonGenerator.writeBoolean((Boolean) value.getValue()); + break; + case LONG: + jsonGenerator.writeNumber((Long) value.getValue()); + break; + case DOUBLE: + jsonGenerator.writeNumber((Double) value.getValue()); + break; + default: + throw new AssertionError("Unknown type: " + columnType); + } + } + } + + public static class Deserializer + extends JsonDeserializer + { + @Override + public SerializableNativeValue deserialize(JsonParser jsonParser, DeserializationContext context) + throws IOException + { + checkJson(jsonParser.nextFieldName(new SerializedString("type"))); + + String typeString = jsonParser.nextTextValue(); + Class type = extractClassType(typeString); + ColumnType.fromNativeType(type); // Make sure the null is a valid type + + checkJson(jsonParser.nextFieldName(new SerializedString("value"))); + + JsonToken token = jsonParser.nextToken(); + Comparable value = (token == JsonToken.VALUE_NULL) ? null : readValue(type, jsonParser); + checkJson(jsonParser.nextToken() == JsonToken.END_OBJECT); + + return new SerializableNativeValue(type, value); + } + + private static Comparable readValue(Class type, JsonParser jsonParser) + throws IOException + { + ColumnType columnType = ColumnType.fromNativeType(type); + switch (columnType) { + case STRING: + String value = jsonParser.getValueAsString(); + checkJson(value != null); + return value; + case BOOLEAN: + return jsonParser.getBooleanValue(); + case LONG: + return jsonParser.getLongValue(); + case DOUBLE: + return jsonParser.getDoubleValue(); + default: + throw new AssertionError("Unknown type: " + columnType); + } + } + + private static void checkJson(boolean condition) + { + if (!condition) { + throw new IllegalArgumentException("Malformed SerializableNativeValue JSON object"); + } + } + + private static Class extractClassType(String typeString) + { + try { + return Class.forName(typeString); + } + catch (ClassNotFoundException e) { + throw new IllegalArgumentException("Unknown class type: " + typeString); + } + } + } +} diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/SortedRangeSet.java b/presto-spi/src/main/java/com/facebook/presto/spi/SortedRangeSet.java new file mode 100644 index 0000000000000..5ae8005c55489 --- /dev/null +++ b/presto-spi/src/main/java/com/facebook/presto/spi/SortedRangeSet.java @@ -0,0 +1,359 @@ +/* + * 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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Objects; +import java.util.TreeMap; + +import static java.util.Collections.unmodifiableCollection; + +/** + * A set containing zero or more Ranges of the same type over a continuous space of possible values. + * Ranges are coalesced into the most compact representation of non-overlapping Ranges. This structure + * allows iteration across these compacted Ranges in increasing order, as well as other common + * set-related operation. + */ +public final class SortedRangeSet + implements Iterable +{ + private final Class type; + private final NavigableMap lowIndexedRanges; + + private SortedRangeSet(Class type, NavigableMap lowIndexedRanges) + { + this.type = Objects.requireNonNull(type, "type is null"); + this.lowIndexedRanges = Objects.requireNonNull(lowIndexedRanges, "lowIndexedRanges is null"); + } + + public static SortedRangeSet none(Class type) + { + return copyOf(type, Collections.emptyList()); + } + + public static SortedRangeSet all(Class type) + { + return copyOf(type, Arrays.asList(Range.all(type))); + } + + public static SortedRangeSet singleValue(Comparable value) + { + return SortedRangeSet.of(Range.equal(value)); + } + + /** + * Provided Ranges are unioned together to form the SortedRangeSet + */ + public static SortedRangeSet of(Range first, Range... ranges) + { + List rangeList = new ArrayList<>(); + rangeList.add(first); + rangeList.addAll(Arrays.asList(ranges)); + return copyOf(first.getType(), rangeList); + } + + /** + * Provided Ranges are unioned together to form the SortedRangeSet + */ + public static SortedRangeSet copyOf(Class type, Iterable ranges) + { + return new Builder(type).addAll(ranges).build(); + } + + @JsonCreator + public static SortedRangeSet copyOf( + @JsonProperty("type") Class type, + @JsonProperty("ranges") List ranges) + { + return copyOf(type, (Iterable) ranges); + } + + @JsonProperty + public Class getType() + { + return type; + } + + @JsonProperty + public List getRanges() + { + ArrayList ranges = new ArrayList<>(); + ranges.addAll(lowIndexedRanges.values()); + return ranges; + } + + @JsonIgnore + public int getRangeCount() + { + return lowIndexedRanges.size(); + } + + @JsonIgnore + public boolean isNone() + { + return lowIndexedRanges.isEmpty(); + } + + @JsonIgnore + public boolean isAll() + { + return lowIndexedRanges.size() == 1 && lowIndexedRanges.values().iterator().next().isAll(); + } + + @JsonIgnore + public boolean isSingleValue() + { + return lowIndexedRanges.size() == 1 && lowIndexedRanges.values().iterator().next().isSingleValue(); + } + + @JsonIgnore + public Comparable getSingleValue() + { + if (!isSingleValue()) { + throw new IllegalStateException("SortedRangeSet does not have just a single value"); + } + return lowIndexedRanges.values().iterator().next().getSingleValue(); + } + + public boolean includesMarker(Marker marker) + { + Objects.requireNonNull(marker, "marker is null"); + checkTypeCompatibility(marker); + Map.Entry floorEntry = lowIndexedRanges.floorEntry(marker); + return floorEntry != null && floorEntry.getValue().includes(marker); + } + + @JsonIgnore + public Range getSpan() + { + if (lowIndexedRanges.isEmpty()) { + throw new IllegalStateException("Can not get span if no ranges exist"); + } + return lowIndexedRanges.firstEntry().getValue().span(lowIndexedRanges.lastEntry().getValue()); + } + + public boolean overlaps(SortedRangeSet other) + { + checkTypeCompatibility(other); + return !this.intersect(other).isNone(); + } + + public boolean contains(SortedRangeSet other) + { + checkTypeCompatibility(other); + return this.union(other).equals(this); + } + + public SortedRangeSet intersect(SortedRangeSet other) + { + checkTypeCompatibility(other); + + Builder builder = new Builder(type); + + Iterator iter1 = iterator(); + Iterator iter2 = other.iterator(); + + if (iter1.hasNext() && iter2.hasNext()) { + Range range1 = iter1.next(); + Range range2 = iter2.next(); + + while (true) { + if (range1.overlaps(range2)) { + builder.add(range1.intersect(range2)); + } + + if (range1.getHigh().compareTo(range2.getHigh()) <= 0) { + if (!iter1.hasNext()) { + break; + } + range1 = iter1.next(); + } + else { + if (!iter2.hasNext()) { + break; + } + range2 = iter2.next(); + } + } + } + + return builder.build(); + } + + public SortedRangeSet union(SortedRangeSet other) + { + checkTypeCompatibility(other); + return new Builder(type) + .addAll(this) + .addAll(other) + .build(); + } + + public SortedRangeSet complement() + { + Builder builder = new Builder(type); + + if (lowIndexedRanges.isEmpty()) { + return builder.add(Range.all(type)).build(); + } + + Iterator rangeIterator = lowIndexedRanges.values().iterator(); + + Range firstRange = rangeIterator.next(); + if (!firstRange.getLow().isLowerUnbounded()) { + builder.add(new Range(Marker.lowerUnbounded(type), firstRange.getLow().lesserAdjacent())); + } + + Range previousRange = firstRange; + while (rangeIterator.hasNext()) { + Range currentRange = rangeIterator.next(); + + Marker lowMarker = previousRange.getHigh().greaterAdjacent(); + Marker highMarker = currentRange.getLow().lesserAdjacent(); + builder.add(new Range(lowMarker, highMarker)); + + previousRange = currentRange; + } + + Range lastRange = previousRange; + if (!lastRange.getHigh().isUpperUnbounded()) { + builder.add(new Range(lastRange.getHigh().greaterAdjacent(), Marker.upperUnbounded(type))); + } + + return builder.build(); + } + + public SortedRangeSet subtract(SortedRangeSet other) + { + checkTypeCompatibility(other); + return this.intersect(other.complement()); + } + + private void checkTypeCompatibility(SortedRangeSet other) + { + if (!getType().equals(other.getType())) { + throw new IllegalStateException(String.format("Mismatched SortedRangeSet types: %s vs %s", getType(), other.getType())); + } + } + + private void checkTypeCompatibility(Marker marker) + { + if (!getType().equals(marker.getType())) { + throw new IllegalStateException(String.format("Marker of %s does not match SortedRangeSet of %s", marker.getType(), getType())); + } + } + + @Override + public Iterator iterator() + { + return unmodifiableCollection(lowIndexedRanges.values()).iterator(); + } + + @Override + public int hashCode() + { + return Objects.hash(lowIndexedRanges); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final SortedRangeSet other = (SortedRangeSet) obj; + return Objects.equals(this.lowIndexedRanges, other.lowIndexedRanges); + } + + @Override + public String toString() + { + return lowIndexedRanges.values().toString(); + } + + public static Builder builder(Class type) + { + return new Builder(type); + } + + public static class Builder + { + private final Class type; + private final NavigableMap lowIndexedRanges = new TreeMap<>(); + + public Builder(Class type) + { + this.type = Objects.requireNonNull(type, "type is null"); + } + + public Builder add(Range range) + { + if (!type.equals(range.getType())) { + throw new IllegalArgumentException(String.format("Range type %s does not match builder type %s", range.getType(), type)); + } + + // Merge with any overlapping ranges + Map.Entry lowFloorEntry = lowIndexedRanges.floorEntry(range.getLow()); + if (lowFloorEntry != null && lowFloorEntry.getValue().overlaps(range)) { + range = lowFloorEntry.getValue().span(range); + } + Map.Entry highFloorEntry = lowIndexedRanges.floorEntry(range.getHigh()); + if (highFloorEntry != null && highFloorEntry.getValue().overlaps(range)) { + range = highFloorEntry.getValue().span(range); + } + + // Merge with any adjacent ranges + if (lowFloorEntry != null && lowFloorEntry.getValue().getHigh().isAdjacent(range.getLow())) { + range = lowFloorEntry.getValue().span(range); + } + Map.Entry highHigherEntry = lowIndexedRanges.higherEntry(range.getHigh()); + if (highHigherEntry != null && highHigherEntry.getValue().getLow().isAdjacent(range.getHigh())) { + range = highHigherEntry.getValue().span(range); + } + + // Delete all encompassed ranges + NavigableMap subMap = lowIndexedRanges.subMap(range.getLow(), true, range.getHigh(), true); + subMap.clear(); + + lowIndexedRanges.put(range.getLow(), range); + return this; + } + + public Builder addAll(Iterable ranges) + { + for (Range range : ranges) { + add(range); + } + return this; + } + + public SortedRangeSet build() + { + return new SortedRangeSet(type, lowIndexedRanges); + } + } +} diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/TupleDomain.java b/presto-spi/src/main/java/com/facebook/presto/spi/TupleDomain.java new file mode 100644 index 0000000000000..1589bf89bc68a --- /dev/null +++ b/presto-spi/src/main/java/com/facebook/presto/spi/TupleDomain.java @@ -0,0 +1,335 @@ +/* + * 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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * Defines a set of valid tuples according to the constraints on each of its constituent columns + */ +public final class TupleDomain +{ + private static final TupleDomain NONE = new TupleDomain(null); + private static final TupleDomain ALL = new TupleDomain(Collections.emptyMap()); + + /** + * TupleDomain is internally represented as a normalized map of each column to its + * respective allowable value Domain. Conceptually, these Domains can be thought of + * as being AND'ed together to form the representative predicate. + * + * This map is normalized in the following ways: + * 1) The map will not contain Domain.none() as any of its values. If any of the Domain + * values are Domain.none(), then the whole map will instead be null. This enforces the fact that + * any single Domain.none() value effectively turns this TupleDomain into "none" as well. + * 2) The map will not contain Domain.all() as any of its values. Our convention here is that + * any unmentioned column is equivalent to having Domain.all(). To normalize this structure, + * we remove any Domain.all() values from the map. + */ + private final Map domains; + + private TupleDomain(Map domains) + { + if (domains == null || containsNoneDomain(domains)) { + this.domains = null; + } + else { + this.domains = Collections.unmodifiableMap(normalizeAndCopy(domains)); + } + } + + public static TupleDomain withColumnDomains(Map domains) + { + return new TupleDomain(Objects.requireNonNull(domains, "domains is null")); + } + + public static TupleDomain none() + { + return NONE; + } + + public static TupleDomain all() + { + return ALL; + } + + /** + * Convert a map of columns to values into the TupleDomain which requires + * those columns to be fixed to those values. + */ + public static TupleDomain withFixedValues(Map> fixedValues) + { + Map domains = new HashMap<>(); + for (Map.Entry> entry : fixedValues.entrySet()) { + domains.put(entry.getKey(), Domain.singleValue(entry.getValue())); + } + return withColumnDomains(domains); + } + + @JsonCreator + // Available for Jackson deserialization only! + public static TupleDomain fromNullableColumnDomains(@JsonProperty("nullableColumnDomains") List nullableColumnDomains) + { + if (nullableColumnDomains == null) { + return none(); + } + return withColumnDomains(toMap(nullableColumnDomains)); + } + + @JsonProperty + // Available for Jackson serialization only! + public List getNullableColumnDomains() + { + return domains == null ? null : toList(domains); + } + + private static Map toMap(List columnDomains) + { + Map map = new HashMap<>(); + for (ColumnDomain columnDomain : columnDomains) { + if (map.containsKey(columnDomain.getColumnHandle())) { + throw new IllegalArgumentException("Duplicate column handle!"); + } + map.put(columnDomain.getColumnHandle(), columnDomain.getDomain()); + } + return map; + } + + private static List toList(Map columnDomains) + { + List list = new ArrayList<>(); + for (Map.Entry entry : columnDomains.entrySet()) { + list.add(new ColumnDomain(entry.getKey(), entry.getValue())); + } + return list; + } + + private static boolean containsNoneDomain(Map domains) + { + for (Domain domain : domains.values()) { + if (domain.isNone()) { + return true; + } + } + return false; + } + + private static Map normalizeAndCopy(Map domains) + { + Map map = new HashMap<>(); + for (Map.Entry entry : domains.entrySet()) { + if (!entry.getValue().isAll()) { + map.put(entry.getKey(), entry.getValue()); + } + } + return map; + } + + /** + * Returns true if any tuples would satisfy this TupleDomain + */ + @JsonIgnore + public boolean isAll() + { + return domains != null && domains.isEmpty(); + } + + /** + * Returns true if no tuple could ever satisfy this TupleDomain + */ + @JsonIgnore + public boolean isNone() + { + return domains == null; + } + + /** + * Gets the TupleDomain as a map of each column to its respective Domain. + * - You must check to make sure that this TupleDomain is not None before calling this method + * - Unmentioned columns have an implicit value of Domain.all() + * - The column Domains can be thought of as AND'ed to together to form the whole predicate + */ + @JsonIgnore + public Map getDomains() + { + if (domains == null) { + throw new IllegalStateException("Can not get column Domains from a none TupleDomain"); + } + return domains; + } + + /** + * Extract all column constraints that require exactly one value in their respective Domains. + */ + public Map> extractFixedValues() + { + if (isNone()) { + return Collections.emptyMap(); + } + + Map> fixedValues = new HashMap<>(); + for (Map.Entry entry : getDomains().entrySet()) { + if (entry.getValue().isSingleValue()) { + fixedValues.put(entry.getKey(), entry.getValue().getSingleValue()); + } + } + return fixedValues; + } + + /** + * Returns the strict intersection of the TupleDomains. + * The resulting TupleDomain represents the set of tuples that would would be valid + * in both TupleDomains. + */ + public TupleDomain intersect(TupleDomain other) + { + if (this.isNone() || other.isNone()) { + return none(); + } + + Map intersected = new HashMap<>(this.getDomains()); + for (Map.Entry entry : other.getDomains().entrySet()) { + Domain intersectionDomain = intersected.get(entry.getKey()); + if (intersectionDomain == null) { + intersected.put(entry.getKey(), entry.getValue()); + } + else { + intersected.put(entry.getKey(), intersectionDomain.intersect(entry.getValue())); + } + } + return withColumnDomains(intersected); + } + + /** + * Returns a TupleDomain in which corresponding column Domains are unioned together. + * + * Note that this is NOT equivalent to a strict union as the final result may allow tuples + * that do not exist in either TupleDomain. + * For example: + * TupleDomain X: a => 1, b => 2 + * TupleDomain Y: a => 2, b => 3 + * Column-wise unioned TupleDomain: a = > 1 OR 2, b => 2 OR 3 + * In the above resulting TupleDomain, tuple (a => 1, b => 3) would be considered valid but would + * not be valid for either TupleDomain X or TupleDomain Y. + * However, this result is guaranteed to be a superset of the strict union. + */ + public TupleDomain columnWiseUnion(TupleDomain other) + { + if (this.isNone()) { + return other; + } + else if (other.isNone()) { + return this; + } + + // Only columns contained in both TupleDomains will make it into the column-wise union. + // This is b/c an unmentioned column is implicitly an "all" Domain and so any union with that "all" Domain will also be the "all" Domain. + Map columnWiseUnioned = new HashMap<>(); + for (Map.Entry entry : this.getDomains().entrySet()) { + Domain otherDomain = other.getDomains().get(entry.getKey()); + if (otherDomain != null) { + columnWiseUnioned.put(entry.getKey(), entry.getValue().union(otherDomain)); + } + } + return withColumnDomains(columnWiseUnioned); + } + + /** + * Returns true only if there exists a strict intersection between the TupleDomains. + * i.e. there exists some potential tuple that would be allowable in both TupleDomains. + */ + public boolean overlaps(TupleDomain other) + { + return !this.intersect(other).isNone(); + } + + /** + * Returns true only if the this TupleDomain contains all possible tuples that would be allowable by + * the other TupleDomain. + */ + public boolean contains(TupleDomain other) + { + return other.isNone() || this.columnWiseUnion(other).equals(this); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (!(o instanceof TupleDomain)) { + return false; + } + + TupleDomain that = (TupleDomain) o; + + if (domains != null ? !domains.equals(that.domains) : that.domains != null) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + return domains != null ? domains.hashCode() : 0; + } + + @Override + public String toString() + { + final StringBuilder sb = new StringBuilder("TupleDomain{"); + sb.append("domains=").append(domains); + sb.append('}'); + return sb.toString(); + } + + // Available for Jackson serialization only! + public static class ColumnDomain + { + private final ColumnHandle columnHandle; + private final Domain domain; + + @JsonCreator + public ColumnDomain( + @JsonProperty("columnHandle") ColumnHandle columnHandle, + @JsonProperty("domain") Domain domain) + { + this.columnHandle = Objects.requireNonNull(columnHandle, "columnHandle is null"); + this.domain = Objects.requireNonNull(domain, "domain is null"); + } + + @JsonProperty + public ColumnHandle getColumnHandle() + { + return columnHandle; + } + + @JsonProperty + public Domain getDomain() + { + return domain; + } + } +} diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/classloader/ClassLoaderSafeConnectorSplitManager.java b/presto-spi/src/main/java/com/facebook/presto/spi/classloader/ClassLoaderSafeConnectorSplitManager.java index ccf33fbfcabe4..2962181b4c072 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/classloader/ClassLoaderSafeConnectorSplitManager.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/classloader/ClassLoaderSafeConnectorSplitManager.java @@ -13,14 +13,14 @@ */ package com.facebook.presto.spi.classloader; -import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.ConnectorSplitManager; import com.facebook.presto.spi.Partition; +import com.facebook.presto.spi.PartitionResult; import com.facebook.presto.spi.Split; import com.facebook.presto.spi.TableHandle; +import com.facebook.presto.spi.TupleDomain; import java.util.List; -import java.util.Map; @SuppressWarnings("UnusedDeclaration") public final class ClassLoaderSafeConnectorSplitManager @@ -52,10 +52,10 @@ public boolean canHandle(TableHandle handle) } @Override - public List getPartitions(TableHandle table, Map bindings) + public PartitionResult getPartitions(TableHandle table, TupleDomain tupleDomain) { try (ThreadContextClassLoader threadContextClassLoader = new ThreadContextClassLoader(classLoader)) { - return delegate.getPartitions(table, bindings); + return delegate.getPartitions(table, tupleDomain); } } diff --git a/presto-spi/src/test/java/com/facebook/presto/spi/TestDomain.java b/presto-spi/src/test/java/com/facebook/presto/spi/TestDomain.java new file mode 100644 index 0000000000000..5db247eb2cc55 --- /dev/null +++ b/presto-spi/src/test/java/com/facebook/presto/spi/TestDomain.java @@ -0,0 +1,380 @@ +/* + * 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; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.testng.Assert; +import org.testng.annotations.Test; + +public class TestDomain +{ + @Test + public void testNone() + throws Exception + { + Domain domain = Domain.none(Long.class); + Assert.assertTrue(domain.isNone()); + Assert.assertFalse(domain.isAll()); + Assert.assertFalse(domain.isSingleValue()); + Assert.assertFalse(domain.isNullAllowed()); + Assert.assertEquals(domain.getRanges(), SortedRangeSet.none(Long.class)); + Assert.assertEquals(domain.getType(), Long.class); + Assert.assertFalse(domain.includesValue(Long.MIN_VALUE)); + Assert.assertFalse(domain.includesValue(0L)); + Assert.assertFalse(domain.includesValue(Long.MAX_VALUE)); + Assert.assertEquals(domain.complement(), Domain.all(Long.class)); + } + + @Test + public void testAll() + throws Exception + { + Domain domain = Domain.all(Long.class); + Assert.assertFalse(domain.isNone()); + Assert.assertTrue(domain.isAll()); + Assert.assertFalse(domain.isSingleValue()); + Assert.assertTrue(domain.isNullAllowed()); + Assert.assertEquals(domain.getRanges(), SortedRangeSet.all(Long.class)); + Assert.assertEquals(domain.getType(), Long.class); + Assert.assertTrue(domain.includesValue(Long.MIN_VALUE)); + Assert.assertTrue(domain.includesValue(0L)); + Assert.assertTrue(domain.includesValue(Long.MAX_VALUE)); + Assert.assertEquals(domain.complement(), Domain.none(Long.class)); + } + + @Test + public void testNullOnly() + throws Exception + { + Domain domain = Domain.onlyNull(Long.class); + Assert.assertFalse(domain.isNone()); + Assert.assertFalse(domain.isAll()); + Assert.assertFalse(domain.isSingleValue()); + Assert.assertTrue(domain.isNullAllowed()); + Assert.assertEquals(domain.getRanges(), SortedRangeSet.none(Long.class)); + Assert.assertEquals(domain.getType(), Long.class); + Assert.assertFalse(domain.includesValue(Long.MIN_VALUE)); + Assert.assertFalse(domain.includesValue(0L)); + Assert.assertFalse(domain.includesValue(Long.MAX_VALUE)); + Assert.assertEquals(domain.complement(), Domain.notNull(Long.class)); + } + + @Test + public void testNotNull() + throws Exception + { + Domain domain = Domain.notNull(Long.class); + Assert.assertFalse(domain.isNone()); + Assert.assertFalse(domain.isAll()); + Assert.assertFalse(domain.isSingleValue()); + Assert.assertFalse(domain.isNullAllowed()); + Assert.assertEquals(domain.getRanges(), SortedRangeSet.all(Long.class)); + Assert.assertEquals(domain.getType(), Long.class); + Assert.assertTrue(domain.includesValue(Long.MIN_VALUE)); + Assert.assertTrue(domain.includesValue(0L)); + Assert.assertTrue(domain.includesValue(Long.MAX_VALUE)); + Assert.assertEquals(domain.complement(), Domain.onlyNull(Long.class)); + } + + @Test + public void testSingleValue() + throws Exception + { + Domain domain = Domain.singleValue(0L); + Assert.assertFalse(domain.isNone()); + Assert.assertFalse(domain.isAll()); + Assert.assertTrue(domain.isSingleValue()); + Assert.assertFalse(domain.isNullAllowed()); + Assert.assertEquals(domain.getRanges(), SortedRangeSet.of(Range.equal(0L))); + Assert.assertEquals(domain.getType(), Long.class); + Assert.assertFalse(domain.includesValue(Long.MIN_VALUE)); + Assert.assertTrue(domain.includesValue(0L)); + Assert.assertFalse(domain.includesValue(Long.MAX_VALUE)); + Assert.assertEquals(domain.complement(), Domain.create(SortedRangeSet.of(Range.lessThan(0L), Range.greaterThan(0L)), true)); + Assert.assertEquals(domain.getSingleValue(), 0L); + + try { + Domain.create(SortedRangeSet.of(Range.range(1, true, 2, true)), false).getSingleValue(); + Assert.fail(); + } + catch (IllegalStateException e) { + } + } + + @Test + public void testOverlaps() + throws Exception + { + Assert.assertTrue(Domain.all(Long.class).overlaps(Domain.all(Long.class))); + Assert.assertFalse(Domain.all(Long.class).overlaps(Domain.none(Long.class))); + Assert.assertTrue(Domain.all(Long.class).overlaps(Domain.notNull(Long.class))); + Assert.assertTrue(Domain.all(Long.class).overlaps(Domain.onlyNull(Long.class))); + Assert.assertTrue(Domain.all(Long.class).overlaps(Domain.singleValue(0L))); + + Assert.assertFalse(Domain.none(Long.class).overlaps(Domain.all(Long.class))); + Assert.assertFalse(Domain.none(Long.class).overlaps(Domain.none(Long.class))); + Assert.assertFalse(Domain.none(Long.class).overlaps(Domain.notNull(Long.class))); + Assert.assertFalse(Domain.none(Long.class).overlaps(Domain.onlyNull(Long.class))); + Assert.assertFalse(Domain.none(Long.class).overlaps(Domain.singleValue(0L))); + + Assert.assertTrue(Domain.notNull(Long.class).overlaps(Domain.all(Long.class))); + Assert.assertFalse(Domain.notNull(Long.class).overlaps(Domain.none(Long.class))); + Assert.assertTrue(Domain.notNull(Long.class).overlaps(Domain.notNull(Long.class))); + Assert.assertFalse(Domain.notNull(Long.class).overlaps(Domain.onlyNull(Long.class))); + Assert.assertTrue(Domain.notNull(Long.class).overlaps(Domain.singleValue(0L))); + + Assert.assertTrue(Domain.onlyNull(Long.class).overlaps(Domain.all(Long.class))); + Assert.assertFalse(Domain.onlyNull(Long.class).overlaps(Domain.none(Long.class))); + Assert.assertFalse(Domain.onlyNull(Long.class).overlaps(Domain.notNull(Long.class))); + Assert.assertTrue(Domain.onlyNull(Long.class).overlaps(Domain.onlyNull(Long.class))); + Assert.assertFalse(Domain.onlyNull(Long.class).overlaps(Domain.singleValue(0L))); + + Assert.assertTrue(Domain.singleValue(0L).overlaps(Domain.all(Long.class))); + Assert.assertFalse(Domain.singleValue(0L).overlaps(Domain.none(Long.class))); + Assert.assertTrue(Domain.singleValue(0L).overlaps(Domain.notNull(Long.class))); + Assert.assertFalse(Domain.singleValue(0L).overlaps(Domain.onlyNull(Long.class))); + Assert.assertTrue(Domain.singleValue(0L).overlaps(Domain.singleValue(0L))); + } + + @Test + public void testContains() + throws Exception + { + Assert.assertTrue(Domain.all(Long.class).contains(Domain.all(Long.class))); + Assert.assertTrue(Domain.all(Long.class).contains(Domain.none(Long.class))); + Assert.assertTrue(Domain.all(Long.class).contains(Domain.notNull(Long.class))); + Assert.assertTrue(Domain.all(Long.class).contains(Domain.onlyNull(Long.class))); + Assert.assertTrue(Domain.all(Long.class).contains(Domain.singleValue(0L))); + + Assert.assertFalse(Domain.none(Long.class).contains(Domain.all(Long.class))); + Assert.assertTrue(Domain.none(Long.class).contains(Domain.none(Long.class))); + Assert.assertFalse(Domain.none(Long.class).contains(Domain.notNull(Long.class))); + Assert.assertFalse(Domain.none(Long.class).contains(Domain.onlyNull(Long.class))); + Assert.assertFalse(Domain.none(Long.class).contains(Domain.singleValue(0L))); + + Assert.assertFalse(Domain.notNull(Long.class).contains(Domain.all(Long.class))); + Assert.assertTrue(Domain.notNull(Long.class).contains(Domain.none(Long.class))); + Assert.assertTrue(Domain.notNull(Long.class).contains(Domain.notNull(Long.class))); + Assert.assertFalse(Domain.notNull(Long.class).contains(Domain.onlyNull(Long.class))); + Assert.assertTrue(Domain.notNull(Long.class).contains(Domain.singleValue(0L))); + + Assert.assertFalse(Domain.onlyNull(Long.class).contains(Domain.all(Long.class))); + Assert.assertTrue(Domain.onlyNull(Long.class).contains(Domain.none(Long.class))); + Assert.assertFalse(Domain.onlyNull(Long.class).contains(Domain.notNull(Long.class))); + Assert.assertTrue(Domain.onlyNull(Long.class).contains(Domain.onlyNull(Long.class))); + Assert.assertFalse(Domain.onlyNull(Long.class).contains(Domain.singleValue(0L))); + + Assert.assertFalse(Domain.singleValue(0L).contains(Domain.all(Long.class))); + Assert.assertTrue(Domain.singleValue(0L).contains(Domain.none(Long.class))); + Assert.assertFalse(Domain.singleValue(0L).contains(Domain.notNull(Long.class))); + Assert.assertFalse(Domain.singleValue(0L).contains(Domain.onlyNull(Long.class))); + Assert.assertTrue(Domain.singleValue(0L).contains(Domain.singleValue(0L))); + } + + @Test + public void testIntersect() + throws Exception + { + Assert.assertEquals( + Domain.all(Long.class).intersect(Domain.all(Long.class)), + Domain.all(Long.class)); + + Assert.assertEquals( + Domain.none(Long.class).intersect(Domain.none(Long.class)), + Domain.none(Long.class)); + + Assert.assertEquals( + Domain.all(Long.class).intersect(Domain.none(Long.class)), + Domain.none(Long.class)); + + Assert.assertEquals( + Domain.notNull(Long.class).intersect(Domain.onlyNull(Long.class)), + Domain.none(Long.class)); + + Assert.assertEquals( + Domain.singleValue(0L).intersect(Domain.all(Long.class)), + Domain.singleValue(0L)); + + Assert.assertEquals( + Domain.singleValue(0L).intersect(Domain.onlyNull(Long.class)), + Domain.none(Long.class)); + + Assert.assertEquals( + Domain.create(SortedRangeSet.of(Range.equal(1L)), true).intersect(Domain.create(SortedRangeSet.of(Range.equal(2L)), true)), + Domain.onlyNull(Long.class)); + + Assert.assertEquals( + Domain.create(SortedRangeSet.of(Range.equal(1L)), true).intersect(Domain.create(SortedRangeSet.of(Range.equal(1L), Range.equal(2L)), false)), + Domain.singleValue(1L)); + } + + @Test + public void testUnion() + throws Exception + { + Assert.assertEquals( + Domain.all(Long.class).union(Domain.all(Long.class)), + Domain.all(Long.class)); + + Assert.assertEquals( + Domain.none(Long.class).union(Domain.none(Long.class)), + Domain.none(Long.class)); + + Assert.assertEquals( + Domain.all(Long.class).union(Domain.none(Long.class)), + Domain.all(Long.class)); + + Assert.assertEquals( + Domain.notNull(Long.class).union(Domain.onlyNull(Long.class)), + Domain.all(Long.class)); + + Assert.assertEquals( + Domain.singleValue(0L).union(Domain.all(Long.class)), + Domain.all(Long.class)); + + Assert.assertEquals( + Domain.singleValue(0L).union(Domain.notNull(Long.class)), + Domain.notNull(Long.class)); + + Assert.assertEquals( + Domain.singleValue(0L).union(Domain.onlyNull(Long.class)), + Domain.create(SortedRangeSet.of(Range.equal(0L)), true)); + + Assert.assertEquals( + Domain.create(SortedRangeSet.of(Range.equal(1L)), true).union(Domain.create(SortedRangeSet.of(Range.equal(2L)), true)), + Domain.create(SortedRangeSet.of(Range.equal(1L), Range.equal(2L)), true)); + + Assert.assertEquals( + Domain.create(SortedRangeSet.of(Range.equal(1L)), true).union(Domain.create(SortedRangeSet.of(Range.equal(1L), Range.equal(2L)), false)), + Domain.create(SortedRangeSet.of(Range.equal(1L), Range.equal(2L)), true)); + } + + @Test + public void testSubtract() + throws Exception + { + Assert.assertEquals( + Domain.all(Long.class).subtract(Domain.all(Long.class)), + Domain.none(Long.class)); + Assert.assertEquals( + Domain.all(Long.class).subtract(Domain.none(Long.class)), + Domain.all(Long.class)); + Assert.assertEquals( + Domain.all(Long.class).subtract(Domain.notNull(Long.class)), + Domain.onlyNull(Long.class)); + Assert.assertEquals( + Domain.all(Long.class).subtract(Domain.onlyNull(Long.class)), + Domain.notNull(Long.class)); + Assert.assertEquals( + Domain.all(Long.class).subtract(Domain.singleValue(0L)), + Domain.create(SortedRangeSet.of(Range.lessThan(0L), Range.greaterThan(0L)), true)); + + Assert.assertEquals( + Domain.none(Long.class).subtract(Domain.all(Long.class)), + Domain.none(Long.class)); + Assert.assertEquals( + Domain.none(Long.class).subtract(Domain.none(Long.class)), + Domain.none(Long.class)); + Assert.assertEquals( + Domain.none(Long.class).subtract(Domain.notNull(Long.class)), + Domain.none(Long.class)); + Assert.assertEquals( + Domain.none(Long.class).subtract(Domain.onlyNull(Long.class)), + Domain.none(Long.class)); + Assert.assertEquals( + Domain.none(Long.class).subtract(Domain.singleValue(0L)), + Domain.none(Long.class)); + + Assert.assertEquals( + Domain.notNull(Long.class).subtract(Domain.all(Long.class)), + Domain.none(Long.class)); + Assert.assertEquals( + Domain.notNull(Long.class).subtract(Domain.none(Long.class)), + Domain.notNull(Long.class)); + Assert.assertEquals( + Domain.notNull(Long.class).subtract(Domain.notNull(Long.class)), + Domain.none(Long.class)); + Assert.assertEquals( + Domain.notNull(Long.class).subtract(Domain.onlyNull(Long.class)), + Domain.notNull(Long.class)); + Assert.assertEquals( + Domain.notNull(Long.class).subtract(Domain.singleValue(0L)), + Domain.create(SortedRangeSet.of(Range.lessThan(0L), Range.greaterThan(0L)), false)); + + Assert.assertEquals( + Domain.onlyNull(Long.class).subtract(Domain.all(Long.class)), + Domain.none(Long.class)); + Assert.assertEquals( + Domain.onlyNull(Long.class).subtract(Domain.none(Long.class)), + Domain.onlyNull(Long.class)); + Assert.assertEquals( + Domain.onlyNull(Long.class).subtract(Domain.notNull(Long.class)), + Domain.onlyNull(Long.class)); + Assert.assertEquals( + Domain.onlyNull(Long.class).subtract(Domain.onlyNull(Long.class)), + Domain.none(Long.class)); + Assert.assertEquals( + Domain.onlyNull(Long.class).subtract(Domain.singleValue(0L)), + Domain.onlyNull(Long.class)); + + Assert.assertEquals( + Domain.singleValue(0L).subtract(Domain.all(Long.class)), + Domain.none(Long.class)); + Assert.assertEquals( + Domain.singleValue(0L).subtract(Domain.none(Long.class)), + Domain.singleValue(0L)); + Assert.assertEquals( + Domain.singleValue(0L).subtract(Domain.notNull(Long.class)), + Domain.none(Long.class)); + Assert.assertEquals( + Domain.singleValue(0L).subtract(Domain.onlyNull(Long.class)), + Domain.singleValue(0L)); + Assert.assertEquals( + Domain.singleValue(0L).subtract(Domain.singleValue(0L)), + Domain.none(Long.class)); + + Assert.assertEquals( + Domain.create(SortedRangeSet.of(Range.equal(1L)), true).subtract(Domain.create(SortedRangeSet.of(Range.equal(2L)), true)), + Domain.singleValue(1L)); + + Assert.assertEquals( + Domain.create(SortedRangeSet.of(Range.equal(1L)), true).subtract(Domain.create(SortedRangeSet.of(Range.equal(1L), Range.equal(2L)), false)), + Domain.onlyNull(Long.class)); + } + + @Test + public void testJsonSerialization() + throws Exception + { + ObjectMapper mapper = new ObjectMapper(); + + Domain domain = Domain.all(Long.class); + Assert.assertEquals(domain, mapper.readValue(mapper.writeValueAsString(domain), Domain.class)); + + domain = Domain.none(Double.class); + Assert.assertEquals(domain, mapper.readValue(mapper.writeValueAsString(domain), Domain.class)); + + domain = Domain.notNull(Boolean.class); + Assert.assertEquals(domain, mapper.readValue(mapper.writeValueAsString(domain), Domain.class)); + + domain = Domain.onlyNull(String.class); + Assert.assertEquals(domain, mapper.readValue(mapper.writeValueAsString(domain), Domain.class)); + + domain = Domain.singleValue(Long.MIN_VALUE); + Assert.assertEquals(domain, mapper.readValue(mapper.writeValueAsString(domain), Domain.class)); + + domain = Domain.create(SortedRangeSet.of(Range.lessThan(0L), Range.equal(1L), Range.range(2L, true, 3L, true)), true); + Assert.assertEquals(domain, mapper.readValue(mapper.writeValueAsString(domain), Domain.class)); + } +} diff --git a/presto-spi/src/test/java/com/facebook/presto/spi/TestMarker.java b/presto-spi/src/test/java/com/facebook/presto/spi/TestMarker.java new file mode 100644 index 0000000000000..b6df1bb5cb8db --- /dev/null +++ b/presto-spi/src/test/java/com/facebook/presto/spi/TestMarker.java @@ -0,0 +1,176 @@ +/* + * 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; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Ordering; +import org.testng.Assert; +import org.testng.annotations.Test; + +import java.util.Map; + +public class TestMarker +{ + @Test + public void testTypes() + throws Exception + { + Assert.assertEquals(Marker.lowerUnbounded(Long.class).getType(), Long.class); + Assert.assertEquals(Marker.below(1L).getType(), Long.class); + Assert.assertEquals(Marker.exactly(1L).getType(), Long.class); + Assert.assertEquals(Marker.above(1L).getType(), Long.class); + Assert.assertEquals(Marker.upperUnbounded(Long.class).getType(), Long.class); + } + + @Test + public void testUnbounded() + throws Exception + { + Assert.assertTrue(Marker.lowerUnbounded(Long.class).isLowerUnbounded()); + Assert.assertFalse(Marker.lowerUnbounded(Long.class).isUpperUnbounded()); + Assert.assertTrue(Marker.upperUnbounded(Long.class).isUpperUnbounded()); + Assert.assertFalse(Marker.upperUnbounded(Long.class).isLowerUnbounded()); + + Assert.assertFalse(Marker.below(1L).isLowerUnbounded()); + Assert.assertFalse(Marker.below(1L).isUpperUnbounded()); + Assert.assertFalse(Marker.exactly(1L).isLowerUnbounded()); + Assert.assertFalse(Marker.exactly(1L).isUpperUnbounded()); + Assert.assertFalse(Marker.above(1L).isLowerUnbounded()); + Assert.assertFalse(Marker.above(1L).isUpperUnbounded()); + } + + @Test + public void testComparisons() + throws Exception + { + ImmutableList markers = ImmutableList.of( + Marker.lowerUnbounded(Long.class), + Marker.above(0L), + Marker.below(1L), + Marker.exactly(1L), + Marker.above(1L), + Marker.below(2L), + Marker.upperUnbounded(Long.class)); + + Assert.assertTrue(Ordering.natural().isStrictlyOrdered(markers)); + + // Compare every marker with every other marker + // Since the markers are strictly ordered, the value of the comparisons should be equivalent to the comparisons + // of their indexes. + for (int i = 0; i < markers.size(); i++) { + for (int j = 0; j < markers.size(); j++) { + Assert.assertTrue(markers.get(i).compareTo(markers.get(j)) == Integer.compare(i, j)); + } + } + } + + @Test + public void testAdjacency() + throws Exception + { + ImmutableMap markers = ImmutableMap.builder() + .put(Marker.lowerUnbounded(Long.class), -1000) + .put(Marker.above(0L), -100) + .put(Marker.below(1L), -1) + .put(Marker.exactly(1L), 0) + .put(Marker.above(1L), 1) + .put(Marker.below(2L), 100) + .put(Marker.upperUnbounded(Long.class), 1000) + .build(); + + // Compare every marker with every other marker + // Map values of distance 1 indicate expected adjacency + for (Map.Entry entry1 : markers.entrySet()) { + for (Map.Entry entry2 : markers.entrySet()) { + boolean adjacent = entry1.getKey().isAdjacent(entry2.getKey()); + boolean distanceIsOne = Math.abs(entry1.getValue() - entry2.getValue()) == 1; + Assert.assertEquals(adjacent, distanceIsOne); + } + } + + Assert.assertEquals(Marker.below(1L).greaterAdjacent(), Marker.exactly(1L)); + Assert.assertEquals(Marker.exactly(1L).greaterAdjacent(), Marker.above(1L)); + Assert.assertEquals(Marker.above(1L).lesserAdjacent(), Marker.exactly(1L)); + Assert.assertEquals(Marker.exactly(1L).lesserAdjacent(), Marker.below(1L)); + + try { + Marker.below(1L).lesserAdjacent(); + Assert.fail(); + } + catch (IllegalStateException e) { + } + + try { + Marker.above(1L).greaterAdjacent(); + Assert.fail(); + } + catch (IllegalStateException e) { + } + + try { + Marker.lowerUnbounded(Long.class).lesserAdjacent(); + Assert.fail(); + } + catch (IllegalStateException e) { + } + + try { + Marker.lowerUnbounded(Long.class).greaterAdjacent(); + Assert.fail(); + } + catch (IllegalStateException e) { + } + + try { + Marker.upperUnbounded(Long.class).lesserAdjacent(); + Assert.fail(); + } + catch (IllegalStateException e) { + } + + try { + Marker.upperUnbounded(Long.class).greaterAdjacent(); + Assert.fail(); + } + catch (IllegalStateException e) { + } + } + + @Test + public void testJsonSerialization() + throws Exception + { + ObjectMapper mapper = new ObjectMapper(); + + Marker marker = Marker.above(0L); + Assert.assertEquals(marker, mapper.readValue(mapper.writeValueAsString(marker), Marker.class)); + + marker = Marker.exactly("abc"); + Assert.assertEquals(marker, mapper.readValue(mapper.writeValueAsString(marker), Marker.class)); + + marker = Marker.below(0.123); + Assert.assertEquals(marker, mapper.readValue(mapper.writeValueAsString(marker), Marker.class)); + + marker = Marker.exactly(true); + Assert.assertEquals(marker, mapper.readValue(mapper.writeValueAsString(marker), Marker.class)); + + marker = Marker.upperUnbounded(Long.class); + Assert.assertEquals(marker, mapper.readValue(mapper.writeValueAsString(marker), Marker.class)); + + marker = Marker.lowerUnbounded(Long.class); + Assert.assertEquals(marker, mapper.readValue(mapper.writeValueAsString(marker), Marker.class)); + } +} diff --git a/presto-spi/src/test/java/com/facebook/presto/spi/TestRange.java b/presto-spi/src/test/java/com/facebook/presto/spi/TestRange.java new file mode 100644 index 0000000000000..d45cbd3eb2db1 --- /dev/null +++ b/presto-spi/src/test/java/com/facebook/presto/spi/TestRange.java @@ -0,0 +1,282 @@ +/* + * 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; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.testng.Assert; +import org.testng.annotations.Test; + +public class TestRange +{ + @SuppressWarnings({ "unchecked", "rawtypes" }) + @Test(expectedExceptions = IllegalArgumentException.class) + public void testMismatchedTypes() + throws Exception + { + // NEVER DO THIS + new Range(Marker.exactly(1L), Marker.exactly("a")); + } + + @Test(expectedExceptions = IllegalArgumentException.class) + public void testInvertedBounds() + throws Exception + { + new Range(Marker.exactly(1L), Marker.exactly(0L)); + } + + @Test(expectedExceptions = IllegalArgumentException.class) + public void testLowerUnboundedOnly() + throws Exception + { + new Range(Marker.lowerUnbounded(Long.class), Marker.lowerUnbounded(Long.class)); + } + + @Test(expectedExceptions = IllegalArgumentException.class) + public void testUpperUnboundedOnly() + throws Exception + { + new Range(Marker.upperUnbounded(Long.class), Marker.upperUnbounded(Long.class)); + } + + @Test + public void testAllRange() + throws Exception + { + Range range = Range.all(Long.class); + Assert.assertEquals(range.getLow(), Marker.lowerUnbounded(Long.class)); + Assert.assertEquals(range.getHigh(), Marker.upperUnbounded(Long.class)); + Assert.assertFalse(range.isSingleValue()); + Assert.assertTrue(range.isAll()); + Assert.assertEquals(range.getType(), Long.class); + Assert.assertTrue(range.includes(Marker.lowerUnbounded(Long.class))); + Assert.assertTrue(range.includes(Marker.below(1L))); + Assert.assertTrue(range.includes(Marker.exactly(1L))); + Assert.assertTrue(range.includes(Marker.above(1L))); + Assert.assertTrue(range.includes(Marker.upperUnbounded(Long.class))); + } + + @Test + public void testGreaterThanRange() + throws Exception + { + Range range = Range.greaterThan(1L); + Assert.assertEquals(range.getLow(), Marker.above(1L)); + Assert.assertEquals(range.getHigh(), Marker.upperUnbounded(Long.class)); + Assert.assertFalse(range.isSingleValue()); + Assert.assertFalse(range.isAll()); + Assert.assertEquals(range.getType(), Long.class); + Assert.assertFalse(range.includes(Marker.lowerUnbounded(Long.class))); + Assert.assertFalse(range.includes(Marker.exactly(1L))); + Assert.assertTrue(range.includes(Marker.exactly(2L))); + Assert.assertTrue(range.includes(Marker.upperUnbounded(Long.class))); + } + + @Test + public void testGreaterThanOrEqualRange() + throws Exception + { + Range range = Range.greaterThanOrEqual(1L); + Assert.assertEquals(range.getLow(), Marker.exactly(1L)); + Assert.assertEquals(range.getHigh(), Marker.upperUnbounded(Long.class)); + Assert.assertFalse(range.isSingleValue()); + Assert.assertFalse(range.isAll()); + Assert.assertEquals(range.getType(), Long.class); + Assert.assertFalse(range.includes(Marker.lowerUnbounded(Long.class))); + Assert.assertFalse(range.includes(Marker.exactly(0L))); + Assert.assertTrue(range.includes(Marker.exactly(1L))); + Assert.assertTrue(range.includes(Marker.exactly(2L))); + Assert.assertTrue(range.includes(Marker.upperUnbounded(Long.class))); + } + + @Test + public void testLessThanRange() + throws Exception + { + Range range = Range.lessThan(1L); + Assert.assertEquals(range.getLow(), Marker.lowerUnbounded(Long.class)); + Assert.assertEquals(range.getHigh(), Marker.below(1L)); + Assert.assertFalse(range.isSingleValue()); + Assert.assertFalse(range.isAll()); + Assert.assertEquals(range.getType(), Long.class); + Assert.assertTrue(range.includes(Marker.lowerUnbounded(Long.class))); + Assert.assertFalse(range.includes(Marker.exactly(1L))); + Assert.assertTrue(range.includes(Marker.exactly(0L))); + Assert.assertFalse(range.includes(Marker.upperUnbounded(Long.class))); + } + + @Test + public void testLessThanOrEqualRange() + throws Exception + { + Range range = Range.lessThanOrEqual(1L); + Assert.assertEquals(range.getLow(), Marker.lowerUnbounded(Long.class)); + Assert.assertEquals(range.getHigh(), Marker.exactly(1L)); + Assert.assertFalse(range.isSingleValue()); + Assert.assertFalse(range.isAll()); + Assert.assertEquals(range.getType(), Long.class); + Assert.assertTrue(range.includes(Marker.lowerUnbounded(Long.class))); + Assert.assertFalse(range.includes(Marker.exactly(2L))); + Assert.assertTrue(range.includes(Marker.exactly(1L))); + Assert.assertTrue(range.includes(Marker.exactly(0L))); + Assert.assertFalse(range.includes(Marker.upperUnbounded(Long.class))); + } + + @Test + public void testEqualRange() + throws Exception + { + Range range = Range.equal(1L); + Assert.assertEquals(range.getLow(), Marker.exactly(1L)); + Assert.assertEquals(range.getHigh(), Marker.exactly(1L)); + Assert.assertTrue(range.isSingleValue()); + Assert.assertFalse(range.isAll()); + Assert.assertEquals(range.getType(), Long.class); + Assert.assertFalse(range.includes(Marker.lowerUnbounded(Long.class))); + Assert.assertFalse(range.includes(Marker.exactly(0L))); + Assert.assertTrue(range.includes(Marker.exactly(1L))); + Assert.assertFalse(range.includes(Marker.exactly(2L))); + Assert.assertFalse(range.includes(Marker.upperUnbounded(Long.class))); + } + + @Test + public void testRange() + throws Exception + { + Range range = Range.range(0L, false, 2L, true); + Assert.assertEquals(range.getLow(), Marker.above(0L)); + Assert.assertEquals(range.getHigh(), Marker.exactly(2L)); + Assert.assertFalse(range.isSingleValue()); + Assert.assertFalse(range.isAll()); + Assert.assertEquals(range.getType(), Long.class); + Assert.assertFalse(range.includes(Marker.lowerUnbounded(Long.class))); + Assert.assertFalse(range.includes(Marker.exactly(0L))); + Assert.assertTrue(range.includes(Marker.exactly(1L))); + Assert.assertTrue(range.includes(Marker.exactly(2L))); + Assert.assertFalse(range.includes(Marker.exactly(3L))); + Assert.assertFalse(range.includes(Marker.upperUnbounded(Long.class))); + } + + @Test + public void testGetSingleValue() + throws Exception + { + Assert.assertEquals(Range.equal(0L).getSingleValue(), 0L); + try { + Range.lessThan(0L).getSingleValue(); + Assert.fail(); + } + catch (IllegalStateException e) { + } + } + + @Test + public void testContains() + throws Exception + { + Assert.assertTrue(Range.all(Long.class).contains(Range.all(Long.class))); + Assert.assertTrue(Range.all(Long.class).contains(Range.equal(0L))); + Assert.assertTrue(Range.all(Long.class).contains(Range.greaterThan(0L))); + Assert.assertTrue(Range.equal(0L).contains(Range.equal(0L))); + Assert.assertFalse(Range.equal(0L).contains(Range.greaterThan(0L))); + Assert.assertFalse(Range.equal(0L).contains(Range.greaterThanOrEqual(0L))); + Assert.assertFalse(Range.equal(0L).contains(Range.all(Long.class))); + Assert.assertTrue(Range.greaterThanOrEqual(0L).contains(Range.greaterThan(0L))); + Assert.assertTrue(Range.greaterThan(0L).contains(Range.greaterThan(1L))); + Assert.assertFalse(Range.greaterThan(0L).contains(Range.lessThan(0L))); + Assert.assertTrue(Range.range(0, true, 2, true).contains(Range.range(1, true, 2, true))); + Assert.assertFalse(Range.range(0, true, 2, true).contains(Range.range(1, true, 3, false))); + } + + @Test + public void testSpan() + throws Exception + { + Assert.assertEquals(Range.greaterThan(1L).span(Range.lessThanOrEqual(2L)), Range.all(Long.class)); + Assert.assertEquals(Range.greaterThan(2L).span(Range.lessThanOrEqual(0L)), Range.all(Long.class)); + Assert.assertEquals(Range.range(1L, true, 3L, false).span(Range.equal(2L)), Range.range(1L, true, 3L, false)); + Assert.assertEquals(Range.range(1L, true, 3L, false).span(Range.range(2L, false, 10L, false)), Range.range(1L, true, 10L, false)); + Assert.assertEquals(Range.greaterThan(1L).span(Range.equal(0L)), Range.greaterThanOrEqual(0L)); + Assert.assertEquals(Range.greaterThan(1L).span(Range.greaterThanOrEqual(10L)), Range.greaterThan(1L)); + Assert.assertEquals(Range.lessThan(1L).span(Range.lessThanOrEqual(1L)), Range.lessThanOrEqual(1L)); + Assert.assertEquals(Range.all(Long.class).span(Range.lessThanOrEqual(1L)), Range.all(Long.class)); + } + + @Test + public void testOverlaps() + throws Exception + { + Assert.assertTrue(Range.greaterThan(1L).overlaps(Range.lessThanOrEqual(2L))); + Assert.assertFalse(Range.greaterThan(2L).overlaps(Range.lessThan(2L))); + Assert.assertTrue(Range.range(1L, true, 3L, false).overlaps(Range.equal(2L))); + Assert.assertTrue(Range.range(1L, true, 3L, false).overlaps(Range.range(2L, false, 10L, false))); + Assert.assertFalse(Range.range(1L, true, 3L, false).overlaps(Range.range(3L, true, 10L, false))); + Assert.assertTrue(Range.range(1L, true, 3L, true).overlaps(Range.range(3L, true, 10L, false))); + Assert.assertTrue(Range.all(Long.class).overlaps(Range.equal(Long.MAX_VALUE))); + } + + @Test + public void testIntersect() + throws Exception + { + Assert.assertEquals(Range.greaterThan(1L).intersect(Range.lessThanOrEqual(2L)), Range.range(1L, false, 2L, true)); + Assert.assertEquals(Range.range(1L, true, 3L, false).intersect(Range.equal(2L)), Range.equal(2L)); + Assert.assertEquals(Range.range(1L, true, 3L, false).intersect(Range.range(2L, false, 10L, false)), Range.range(2L, false, 3L, false)); + Assert.assertEquals(Range.range(1L, true, 3L, true).intersect(Range.range(3L, true, 10L, false)), Range.equal(3L)); + Assert.assertEquals(Range.all(Long.class).intersect(Range.equal(Long.MAX_VALUE)), Range.equal(Long.MAX_VALUE)); + } + + @Test + public void testExceptionalIntersect() + throws Exception + { + try { + Range.greaterThan(2L).intersect(Range.lessThan(2L)); + Assert.fail(); + } + catch (IllegalArgumentException e) { + } + + try { + Range.range(1L, true, 3L, false).intersect(Range.range(3L, true, 10L, false)); + Assert.fail(); + } + catch (IllegalArgumentException e) { + } + } + + @Test + public void testJsonSerialization() + throws Exception + { + ObjectMapper mapper = new ObjectMapper(); + + Range range = Range.all(Long.class); + Assert.assertEquals(range, mapper.readValue(mapper.writeValueAsString(range), Range.class)); + + range = Range.equal(0.123); + Assert.assertEquals(range, mapper.readValue(mapper.writeValueAsString(range), Range.class)); + + range = Range.greaterThan(0L); + Assert.assertEquals(range, mapper.readValue(mapper.writeValueAsString(range), Range.class)); + + range = Range.greaterThanOrEqual("abc"); + Assert.assertEquals(range, mapper.readValue(mapper.writeValueAsString(range), Range.class)); + + range = Range.lessThan(Long.MAX_VALUE); + Assert.assertEquals(range, mapper.readValue(mapper.writeValueAsString(range), Range.class)); + + range = Range.lessThanOrEqual(Double.MAX_VALUE); + Assert.assertEquals(range, mapper.readValue(mapper.writeValueAsString(range), Range.class)); + } +} diff --git a/presto-spi/src/test/java/com/facebook/presto/spi/TestSortedRangeSet.java b/presto-spi/src/test/java/com/facebook/presto/spi/TestSortedRangeSet.java new file mode 100644 index 0000000000000..bc78a5da9ce42 --- /dev/null +++ b/presto-spi/src/test/java/com/facebook/presto/spi/TestSortedRangeSet.java @@ -0,0 +1,457 @@ +/* + * 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; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import org.testng.Assert; +import org.testng.annotations.Test; + +import java.util.Iterator; + +public class TestSortedRangeSet +{ + @Test + public void testEmptySet() + throws Exception + { + SortedRangeSet rangeSet = SortedRangeSet.none(Long.class); + Assert.assertEquals(rangeSet.getType(), Long.class); + Assert.assertTrue(rangeSet.isNone()); + Assert.assertFalse(rangeSet.isAll()); + Assert.assertFalse(rangeSet.isSingleValue()); + Assert.assertTrue(Iterables.isEmpty(rangeSet)); + Assert.assertEquals(rangeSet.getRangeCount(), 0); + Assert.assertEquals(rangeSet.getType(), Long.class); + Assert.assertEquals(rangeSet.complement(), SortedRangeSet.all(Long.class)); + Assert.assertFalse(rangeSet.includesMarker(Marker.lowerUnbounded(Long.class))); + Assert.assertFalse(rangeSet.includesMarker(Marker.exactly(0L))); + Assert.assertFalse(rangeSet.includesMarker(Marker.upperUnbounded(Long.class))); + } + + @Test + public void testEntireSet() + throws Exception + { + SortedRangeSet rangeSet = SortedRangeSet.all(Long.class); + Assert.assertEquals(rangeSet.getType(), Long.class); + Assert.assertFalse(rangeSet.isNone()); + Assert.assertTrue(rangeSet.isAll()); + Assert.assertFalse(rangeSet.isSingleValue()); + Assert.assertEquals(rangeSet.getRangeCount(), 1); + Assert.assertEquals(rangeSet.getType(), Long.class); + Assert.assertEquals(rangeSet.complement(), SortedRangeSet.none(Long.class)); + Assert.assertTrue(rangeSet.includesMarker(Marker.lowerUnbounded(Long.class))); + Assert.assertTrue(rangeSet.includesMarker(Marker.exactly(0L))); + Assert.assertTrue(rangeSet.includesMarker(Marker.upperUnbounded(Long.class))); + } + + @Test + public void testSingleValue() + throws Exception + { + SortedRangeSet rangeSet = SortedRangeSet.singleValue(10L); + + SortedRangeSet complement = SortedRangeSet.of(Range.greaterThan(10L), Range.lessThan(10L)); + + Assert.assertEquals(rangeSet.getType(), Long.class); + Assert.assertFalse(rangeSet.isNone()); + Assert.assertFalse(rangeSet.isAll()); + Assert.assertTrue(rangeSet.isSingleValue()); + Assert.assertTrue(Iterables.elementsEqual(rangeSet, ImmutableList.of(Range.equal(10L)))); + Assert.assertEquals(rangeSet.getRangeCount(), 1); + Assert.assertEquals(rangeSet.getType(), Long.class); + Assert.assertEquals(rangeSet.complement(), complement); + Assert.assertFalse(rangeSet.includesMarker(Marker.lowerUnbounded(Long.class))); + Assert.assertTrue(rangeSet.includesMarker(Marker.exactly(10L))); + Assert.assertFalse(rangeSet.includesMarker(Marker.exactly(9L))); + Assert.assertFalse(rangeSet.includesMarker(Marker.upperUnbounded(Long.class))); + } + + @Test + public void testBoundedSet() + throws Exception + { + SortedRangeSet rangeSet = SortedRangeSet.of( + Range.equal(10L), + Range.equal(0L), + Range.range(9L, true, 11L, false), + Range.equal(0L), + Range.range(2L, true, 4L, true), + Range.range(4L, false, 5L, true)); + + ImmutableList normalizedResult = ImmutableList.of( + Range.equal(0L), + Range.range(2L, true, 5L, true), + Range.range(9L, true, 11L, false)); + + SortedRangeSet complement = SortedRangeSet.of( + Range.lessThan(0L), + Range.range(0L, false, 2L, false), + Range.range(5L, false, 9L, false), + Range.greaterThanOrEqual(11L)); + + Assert.assertEquals(rangeSet.getType(), Long.class); + Assert.assertFalse(rangeSet.isNone()); + Assert.assertFalse(rangeSet.isAll()); + Assert.assertFalse(rangeSet.isSingleValue()); + Assert.assertTrue(Iterables.elementsEqual(rangeSet, normalizedResult)); + Assert.assertEquals(rangeSet, SortedRangeSet.copyOf(Long.class, normalizedResult)); + Assert.assertEquals(rangeSet.getRangeCount(), 3); + Assert.assertEquals(rangeSet.getType(), Long.class); + Assert.assertEquals(rangeSet.complement(), complement); + Assert.assertFalse(rangeSet.includesMarker(Marker.lowerUnbounded(Long.class))); + Assert.assertTrue(rangeSet.includesMarker(Marker.exactly(0L))); + Assert.assertFalse(rangeSet.includesMarker(Marker.exactly(1L))); + Assert.assertFalse(rangeSet.includesMarker(Marker.exactly(7L))); + Assert.assertTrue(rangeSet.includesMarker(Marker.exactly(9L))); + Assert.assertFalse(rangeSet.includesMarker(Marker.upperUnbounded(Long.class))); + } + + @Test + public void testUnboundedSet() + throws Exception + { + SortedRangeSet rangeSet = SortedRangeSet.of( + Range.greaterThan(10L), + Range.lessThanOrEqual(0L), + Range.range(2L, true, 4L, false), + Range.range(4L, true, 6L, false), + Range.range(1L, false, 2L, false), + Range.range(9L, false, 11L, false)); + + ImmutableList normalizedResult = ImmutableList.of( + Range.lessThanOrEqual(0L), + Range.range(1L, false, 6L, false), + Range.greaterThan(9L)); + + SortedRangeSet complement = SortedRangeSet.of( + Range.range(0L, false, 1L, true), + Range.range(6L, true, 9L, true)); + + Assert.assertEquals(rangeSet.getType(), Long.class); + Assert.assertFalse(rangeSet.isNone()); + Assert.assertFalse(rangeSet.isAll()); + Assert.assertFalse(rangeSet.isSingleValue()); + Assert.assertTrue(Iterables.elementsEqual(rangeSet, normalizedResult)); + Assert.assertEquals(rangeSet, SortedRangeSet.copyOf(Long.class, normalizedResult)); + Assert.assertEquals(rangeSet.getRangeCount(), 3); + Assert.assertEquals(rangeSet.getType(), Long.class); + Assert.assertEquals(rangeSet.complement(), complement); + Assert.assertTrue(rangeSet.includesMarker(Marker.lowerUnbounded(Long.class))); + Assert.assertTrue(rangeSet.includesMarker(Marker.exactly(0L))); + Assert.assertTrue(rangeSet.includesMarker(Marker.exactly(4L))); + Assert.assertFalse(rangeSet.includesMarker(Marker.exactly(7L))); + Assert.assertTrue(rangeSet.includesMarker(Marker.upperUnbounded(Long.class))); + } + + @Test + public void testGetSingleValue() + throws Exception + { + Assert.assertEquals(SortedRangeSet.singleValue(0L).getSingleValue(), 0L); + try { + SortedRangeSet.all(Long.class).getSingleValue(); + Assert.fail(); + } + catch (IllegalStateException e) { + } + } + + @Test + public void testSpan() + throws Exception + { + try { + SortedRangeSet.none(Long.class).getSpan(); + Assert.fail(); + } + catch (IllegalStateException e) { + } + + Assert.assertEquals(SortedRangeSet.all(Long.class).getSpan(), Range.all(Long.class)); + Assert.assertEquals(SortedRangeSet.singleValue(0L).getSpan(), Range.equal(0L)); + Assert.assertEquals(SortedRangeSet.of(Range.equal(0L), Range.equal(1L)).getSpan(), Range.range(0L, true, 1L, true)); + Assert.assertEquals(SortedRangeSet.of(Range.equal(0L), Range.greaterThan(1L)).getSpan(), Range.greaterThanOrEqual(0L)); + Assert.assertEquals(SortedRangeSet.of(Range.lessThan(0L), Range.greaterThan(1L)).getSpan(), Range.all(Long.class)); + } + + @Test + public void testOverlaps() + throws Exception + { + Assert.assertTrue(SortedRangeSet.all(Long.class).overlaps(SortedRangeSet.all(Long.class))); + Assert.assertFalse(SortedRangeSet.all(Long.class).overlaps(SortedRangeSet.none(Long.class))); + Assert.assertTrue(SortedRangeSet.all(Long.class).overlaps(SortedRangeSet.singleValue(0L))); + Assert.assertTrue(SortedRangeSet.all(Long.class).overlaps(SortedRangeSet.of(Range.equal(0L), Range.equal(1L)))); + Assert.assertTrue(SortedRangeSet.all(Long.class).overlaps(SortedRangeSet.of(Range.greaterThan(0L)))); + Assert.assertTrue(SortedRangeSet.all(Long.class).overlaps(SortedRangeSet.of(Range.greaterThan(0L), Range.lessThan(0L)))); + + Assert.assertFalse(SortedRangeSet.none(Long.class).overlaps(SortedRangeSet.all(Long.class))); + Assert.assertFalse(SortedRangeSet.none(Long.class).overlaps(SortedRangeSet.none(Long.class))); + Assert.assertFalse(SortedRangeSet.none(Long.class).overlaps(SortedRangeSet.singleValue(0L))); + Assert.assertFalse(SortedRangeSet.none(Long.class).overlaps(SortedRangeSet.of(Range.equal(0L), Range.equal(1L)))); + Assert.assertFalse(SortedRangeSet.none(Long.class).overlaps(SortedRangeSet.of(Range.greaterThan(0L)))); + Assert.assertFalse(SortedRangeSet.none(Long.class).overlaps(SortedRangeSet.of(Range.greaterThan(0L), Range.lessThan(0L)))); + + Assert.assertTrue(SortedRangeSet.singleValue(0L).overlaps(SortedRangeSet.all(Long.class))); + Assert.assertFalse(SortedRangeSet.singleValue(0L).overlaps(SortedRangeSet.none(Long.class))); + Assert.assertTrue(SortedRangeSet.singleValue(0L).overlaps(SortedRangeSet.singleValue(0L))); + Assert.assertTrue(SortedRangeSet.singleValue(0L).overlaps(SortedRangeSet.of(Range.equal(0L), Range.equal(1L)))); + Assert.assertFalse(SortedRangeSet.singleValue(0L).overlaps(SortedRangeSet.of(Range.greaterThan(0L)))); + Assert.assertFalse(SortedRangeSet.singleValue(0L).overlaps(SortedRangeSet.of(Range.greaterThan(0L), Range.lessThan(0L)))); + + Assert.assertTrue(SortedRangeSet.of(Range.equal(0L), Range.equal(1L)).overlaps(SortedRangeSet.of(Range.equal(1L)))); + Assert.assertFalse(SortedRangeSet.of(Range.equal(0L), Range.equal(1L)).overlaps(SortedRangeSet.of(Range.equal(2L)))); + Assert.assertTrue(SortedRangeSet.of(Range.greaterThanOrEqual(0L)).overlaps(SortedRangeSet.of(Range.greaterThan(0L)))); + Assert.assertTrue(SortedRangeSet.of(Range.greaterThan(0L)).overlaps(SortedRangeSet.of(Range.greaterThanOrEqual(0L)))); + Assert.assertFalse(SortedRangeSet.of(Range.lessThan(0L)).overlaps(SortedRangeSet.of(Range.greaterThan(0L)))); + } + + @Test + public void testContains() + throws Exception + { + Assert.assertTrue(SortedRangeSet.all(Long.class).contains(SortedRangeSet.all(Long.class))); + Assert.assertTrue(SortedRangeSet.all(Long.class).contains(SortedRangeSet.none(Long.class))); + Assert.assertTrue(SortedRangeSet.all(Long.class).contains(SortedRangeSet.singleValue(0L))); + Assert.assertTrue(SortedRangeSet.all(Long.class).contains(SortedRangeSet.of(Range.equal(0L), Range.equal(1L)))); + Assert.assertTrue(SortedRangeSet.all(Long.class).contains(SortedRangeSet.of(Range.greaterThan(0L)))); + Assert.assertTrue(SortedRangeSet.all(Long.class).contains(SortedRangeSet.of(Range.greaterThan(0L), Range.lessThan(0L)))); + + Assert.assertFalse(SortedRangeSet.none(Long.class).contains(SortedRangeSet.all(Long.class))); + Assert.assertTrue(SortedRangeSet.none(Long.class).contains(SortedRangeSet.none(Long.class))); + Assert.assertFalse(SortedRangeSet.none(Long.class).contains(SortedRangeSet.singleValue(0L))); + Assert.assertFalse(SortedRangeSet.none(Long.class).contains(SortedRangeSet.of(Range.equal(0L), Range.equal(1L)))); + Assert.assertFalse(SortedRangeSet.none(Long.class).contains(SortedRangeSet.of(Range.greaterThan(0L)))); + Assert.assertFalse(SortedRangeSet.none(Long.class).contains(SortedRangeSet.of(Range.greaterThan(0L), Range.lessThan(0L)))); + + Assert.assertFalse(SortedRangeSet.singleValue(0L).contains(SortedRangeSet.all(Long.class))); + Assert.assertTrue(SortedRangeSet.singleValue(0L).contains(SortedRangeSet.none(Long.class))); + Assert.assertTrue(SortedRangeSet.singleValue(0L).contains(SortedRangeSet.singleValue(0L))); + Assert.assertFalse(SortedRangeSet.singleValue(0L).contains(SortedRangeSet.of(Range.equal(0L), Range.equal(1L)))); + Assert.assertFalse(SortedRangeSet.singleValue(0L).contains(SortedRangeSet.of(Range.greaterThan(0L)))); + Assert.assertFalse(SortedRangeSet.singleValue(0L).contains(SortedRangeSet.of(Range.greaterThan(0L), Range.lessThan(0L)))); + + Assert.assertTrue(SortedRangeSet.of(Range.equal(0L), Range.equal(1L)).contains(SortedRangeSet.of(Range.equal(1L)))); + Assert.assertFalse(SortedRangeSet.of(Range.equal(0L), Range.equal(1L)).contains(SortedRangeSet.of(Range.equal(1L), Range.equal(2L)))); + Assert.assertTrue(SortedRangeSet.of(Range.greaterThanOrEqual(0L)).contains(SortedRangeSet.of(Range.greaterThan(0L)))); + Assert.assertFalse(SortedRangeSet.of(Range.greaterThan(0L)).contains(SortedRangeSet.of(Range.greaterThanOrEqual(0L)))); + Assert.assertFalse(SortedRangeSet.of(Range.lessThan(0L)).contains(SortedRangeSet.of(Range.greaterThan(0L)))); + } + + @Test + public void testIntersect() + throws Exception + { + Assert.assertEquals( + SortedRangeSet.none(Long.class).intersect( + SortedRangeSet.none(Long.class)), + SortedRangeSet.none(Long.class)); + + Assert.assertEquals( + SortedRangeSet.all(Long.class).intersect( + SortedRangeSet.all(Long.class)), + SortedRangeSet.all(Long.class)); + + Assert.assertEquals( + SortedRangeSet.none(Long.class).intersect( + SortedRangeSet.all(Long.class)), + SortedRangeSet.none(Long.class)); + + Assert.assertEquals( + SortedRangeSet.of(Range.equal(1L), Range.equal(2L), Range.equal(3L)).intersect( + SortedRangeSet.of(Range.equal(2L), Range.equal(4L))), + SortedRangeSet.of(Range.equal(2L))); + + Assert.assertEquals( + SortedRangeSet.all(Long.class).intersect( + SortedRangeSet.of(Range.equal(2L), Range.equal(4L))), + SortedRangeSet.of(Range.equal(2L), Range.equal(4L))); + + Assert.assertEquals( + SortedRangeSet.of(Range.range(0L, true, 4L, false)).intersect( + SortedRangeSet.of(Range.equal(2L), Range.greaterThan(3L))), + SortedRangeSet.of(Range.equal(2L), Range.range(3L, false, 4L, false))); + + Assert.assertEquals( + SortedRangeSet.of(Range.greaterThanOrEqual(0L)).intersect( + SortedRangeSet.of(Range.lessThanOrEqual(0L))), + SortedRangeSet.of(Range.equal(0L))); + + Assert.assertEquals( + SortedRangeSet.of(Range.greaterThanOrEqual(-1L)).intersect( + SortedRangeSet.of(Range.lessThanOrEqual(1L))), + SortedRangeSet.of(Range.range(-1L, true, 1L, true))); + } + + @Test + public void testUnion() + throws Exception + { + Assert.assertEquals( + SortedRangeSet.none(Long.class).union( + SortedRangeSet.none(Long.class)), + SortedRangeSet.none(Long.class)); + + Assert.assertEquals( + SortedRangeSet.all(Long.class).union( + SortedRangeSet.all(Long.class)), + SortedRangeSet.all(Long.class)); + + Assert.assertEquals( + SortedRangeSet.none(Long.class).union( + SortedRangeSet.all(Long.class)), + SortedRangeSet.all(Long.class)); + + Assert.assertEquals( + SortedRangeSet.of(Range.equal(1L), Range.equal(2L)).union( + SortedRangeSet.of(Range.equal(2L), Range.equal(3L))), + SortedRangeSet.of(Range.equal(1L), Range.equal(2L), Range.equal(3L))); + + Assert.assertEquals( + SortedRangeSet.all(Long.class).union( + SortedRangeSet.of(Range.equal(0L))), + SortedRangeSet.all(Long.class)); + + Assert.assertEquals( + SortedRangeSet.of(Range.range(0L, true, 4L, false)).union( + SortedRangeSet.of(Range.greaterThan(3L))), + SortedRangeSet.of(Range.greaterThanOrEqual(0L))); + + Assert.assertEquals( + SortedRangeSet.of(Range.greaterThanOrEqual(0L)).union( + SortedRangeSet.of(Range.lessThanOrEqual(0L))), + SortedRangeSet.of(Range.all(Long.class))); + + Assert.assertEquals( + SortedRangeSet.of(Range.greaterThan(0L)).union( + SortedRangeSet.of(Range.lessThan(0L))), + SortedRangeSet.singleValue(0L).complement()); + } + + @Test + public void testSubtract() + throws Exception + { + Assert.assertEquals( + SortedRangeSet.all(Long.class).subtract(SortedRangeSet.all(Long.class)), + SortedRangeSet.none(Long.class)); + Assert.assertEquals( + SortedRangeSet.all(Long.class).subtract(SortedRangeSet.none(Long.class)), + SortedRangeSet.all(Long.class)); + Assert.assertEquals( + SortedRangeSet.all(Long.class).subtract(SortedRangeSet.singleValue(0L)), + SortedRangeSet.singleValue(0L).complement()); + Assert.assertEquals( + SortedRangeSet.all(Long.class).subtract(SortedRangeSet.of(Range.equal(0L), Range.equal(1L))), + SortedRangeSet.of(Range.equal(0L), Range.equal(1L)).complement()); + Assert.assertEquals( + SortedRangeSet.all(Long.class).subtract(SortedRangeSet.of(Range.greaterThan(0L))), + SortedRangeSet.of(Range.lessThanOrEqual(0L))); + + Assert.assertEquals( + SortedRangeSet.none(Long.class).subtract(SortedRangeSet.all(Long.class)), + SortedRangeSet.none(Long.class)); + Assert.assertEquals( + SortedRangeSet.none(Long.class).subtract(SortedRangeSet.none(Long.class)), + SortedRangeSet.none(Long.class)); + Assert.assertEquals( + SortedRangeSet.none(Long.class).subtract(SortedRangeSet.singleValue(0L)), + SortedRangeSet.none(Long.class)); + Assert.assertEquals( + SortedRangeSet.none(Long.class).subtract(SortedRangeSet.of(Range.equal(0L), Range.equal(1L))), + SortedRangeSet.none(Long.class)); + Assert.assertEquals( + SortedRangeSet.none(Long.class).subtract(SortedRangeSet.of(Range.greaterThan(0L))), + SortedRangeSet.none(Long.class)); + + Assert.assertEquals( + SortedRangeSet.singleValue(0L).subtract(SortedRangeSet.all(Long.class)), + SortedRangeSet.none(Long.class)); + Assert.assertEquals( + SortedRangeSet.singleValue(0L).subtract(SortedRangeSet.none(Long.class)), + SortedRangeSet.singleValue(0L)); + Assert.assertEquals( + SortedRangeSet.singleValue(0L).subtract(SortedRangeSet.singleValue(0L)), + SortedRangeSet.none(Long.class)); + Assert.assertEquals( + SortedRangeSet.singleValue(0L).subtract(SortedRangeSet.of(Range.equal(0L), Range.equal(1L))), + SortedRangeSet.none(Long.class)); + Assert.assertEquals( + SortedRangeSet.singleValue(0L).subtract(SortedRangeSet.of(Range.greaterThan(0L))), + SortedRangeSet.singleValue(0L)); + + Assert.assertEquals( + SortedRangeSet.of(Range.equal(0L), Range.equal(1L)).subtract(SortedRangeSet.all(Long.class)), + SortedRangeSet.none(Long.class)); + Assert.assertEquals( + SortedRangeSet.of(Range.equal(0L), Range.equal(1L)).subtract(SortedRangeSet.none(Long.class)), + SortedRangeSet.of(Range.equal(0L), Range.equal(1L))); + Assert.assertEquals( + SortedRangeSet.of(Range.equal(0L), Range.equal(1L)).subtract(SortedRangeSet.singleValue(0L)), + SortedRangeSet.singleValue(1L)); + Assert.assertEquals( + SortedRangeSet.of(Range.equal(0L), Range.equal(1L)).subtract(SortedRangeSet.of(Range.equal(0L), Range.equal(1L))), + SortedRangeSet.none(Long.class)); + Assert.assertEquals( + SortedRangeSet.of(Range.equal(0L), Range.equal(1L)).subtract(SortedRangeSet.of(Range.greaterThan(0L))), + SortedRangeSet.of(Range.equal(0L))); + + Assert.assertEquals( + SortedRangeSet.of(Range.greaterThan(0L)).subtract(SortedRangeSet.all(Long.class)), + SortedRangeSet.none(Long.class)); + Assert.assertEquals( + SortedRangeSet.of(Range.greaterThan(0L)).subtract(SortedRangeSet.none(Long.class)), + SortedRangeSet.of(Range.greaterThan(0L))); + Assert.assertEquals( + SortedRangeSet.of(Range.greaterThan(0L)).subtract(SortedRangeSet.singleValue(0L)), + SortedRangeSet.of(Range.greaterThan(0L))); + Assert.assertEquals( + SortedRangeSet.of(Range.greaterThan(0L)).subtract(SortedRangeSet.of(Range.equal(0L), Range.equal(1L))), + SortedRangeSet.of(Range.range(0L, false, 1L, false), Range.greaterThan(1L))); + Assert.assertEquals( + SortedRangeSet.of(Range.greaterThan(0L)).subtract(SortedRangeSet.of(Range.greaterThan(0L))), + SortedRangeSet.none(Long.class)); + } + + @Test(expectedExceptions = UnsupportedOperationException.class) + public void testUnmodifiableIterator() + throws Exception + { + Iterator iterator = SortedRangeSet.of(Range.equal(1L)).iterator(); + iterator.next(); + iterator.remove(); + } + + @Test + public void testJsonSerialization() + throws Exception + { + ObjectMapper mapper = new ObjectMapper(); + + SortedRangeSet set = SortedRangeSet.all(Long.class); + Assert.assertEquals(set, mapper.readValue(mapper.writeValueAsString(set), SortedRangeSet.class)); + + set = SortedRangeSet.none(Double.class); + Assert.assertEquals(set, mapper.readValue(mapper.writeValueAsString(set), SortedRangeSet.class)); + + set = SortedRangeSet.singleValue("abc"); + Assert.assertEquals(set, mapper.readValue(mapper.writeValueAsString(set), SortedRangeSet.class)); + + set = SortedRangeSet.of(Range.equal(true), Range.equal(false)); + Assert.assertEquals(set, mapper.readValue(mapper.writeValueAsString(set), SortedRangeSet.class)); + } +} diff --git a/presto-spi/src/test/java/com/facebook/presto/spi/TestTupleDomain.java b/presto-spi/src/test/java/com/facebook/presto/spi/TestTupleDomain.java new file mode 100644 index 0000000000000..62ed09315ce75 --- /dev/null +++ b/presto-spi/src/test/java/com/facebook/presto/spi/TestTupleDomain.java @@ -0,0 +1,613 @@ +/* + * 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; + +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonDeserializer; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.common.collect.ImmutableMap; +import org.testng.Assert; +import org.testng.annotations.Test; + +import java.io.IOException; +import java.util.Map; + +public class TestTupleDomain +{ + private static final ColumnHandle A = new TestingColumnHandle("a"); + private static final ColumnHandle B = new TestingColumnHandle("b"); + private static final ColumnHandle C = new TestingColumnHandle("c"); + private static final ColumnHandle D = new TestingColumnHandle("d"); + private static final ColumnHandle E = new TestingColumnHandle("e"); + private static final ColumnHandle F = new TestingColumnHandle("f"); + + @Test + public void testNone() + throws Exception + { + Assert.assertTrue(TupleDomain.none().isNone()); + Assert.assertEquals(TupleDomain.none(), + TupleDomain.withColumnDomains(ImmutableMap.of( + A, Domain.none(Long.class)))); + Assert.assertEquals(TupleDomain.none(), + TupleDomain.withColumnDomains(ImmutableMap.of( + A, Domain.all(Long.class), + B, Domain.none(String.class)))); + } + + @Test + public void testAll() + throws Exception + { + Assert.assertTrue(TupleDomain.all().isAll()); + Assert.assertEquals(TupleDomain.all(), + TupleDomain.withColumnDomains(ImmutableMap.of( + A, Domain.all(Long.class)))); + Assert.assertEquals(TupleDomain.all(), + TupleDomain.withColumnDomains(ImmutableMap.of())); + } + + @Test + public void testIntersection() + throws Exception + { + TupleDomain tupleDomain1 = TupleDomain.withColumnDomains( + ImmutableMap.builder() + .put(A, Domain.all(String.class)) + .put(B, Domain.notNull(Double.class)) + .put(C, Domain.singleValue(1L)) + .put(D, Domain.create(SortedRangeSet.of(Range.greaterThanOrEqual(0.0)), true)) + .build()); + + TupleDomain tupleDomain2 = TupleDomain.withColumnDomains( + ImmutableMap.builder() + .put(A, Domain.singleValue("value")) + .put(B, Domain.singleValue(0.0)) + .put(C, Domain.singleValue(1L)) + .put(D, Domain.create(SortedRangeSet.of(Range.lessThan(10.0)), false)) + .build()); + + TupleDomain expectedTupleDomain = TupleDomain.withColumnDomains( + ImmutableMap.builder() + .put(A, Domain.singleValue("value")) + .put(B, Domain.singleValue(0.0)) + .put(C, Domain.singleValue(1L)) + .put(D, Domain.create(SortedRangeSet.of(Range.range(0.0, true, 10.0, false)), false)) + .build()); + + Assert.assertEquals(tupleDomain1.intersect(tupleDomain2), expectedTupleDomain); + } + + @Test + public void testNoneIntersection() + throws Exception + { + Assert.assertEquals(TupleDomain.none().intersect(TupleDomain.all()), TupleDomain.none()); + Assert.assertEquals(TupleDomain.all().intersect(TupleDomain.none()), TupleDomain.none()); + Assert.assertEquals(TupleDomain.none().intersect(TupleDomain.none()), TupleDomain.none()); + Assert.assertEquals(TupleDomain.withColumnDomains( + ImmutableMap.of(A, Domain.onlyNull(Long.class))) + .intersect( + TupleDomain.withColumnDomains(ImmutableMap.of(A, Domain.notNull(Long.class)))), + TupleDomain.none()); + } + + @Test + public void testMismatchedColumnIntersection() + throws Exception + { + TupleDomain tupleDomain1 = TupleDomain.withColumnDomains( + ImmutableMap.of( + A, Domain.all(Double.class), + B, Domain.singleValue("value"))); + + TupleDomain tupleDomain2 = TupleDomain.withColumnDomains( + ImmutableMap.of( + A, Domain.create(SortedRangeSet.of(Range.greaterThanOrEqual(0.0)), true), + C, Domain.singleValue(1L))); + + TupleDomain expectedTupleDomain = TupleDomain.withColumnDomains(ImmutableMap.of( + A, Domain.create(SortedRangeSet.of(Range.greaterThanOrEqual(0.0)), true), + B, Domain.singleValue("value"), + C, Domain.singleValue(1L))); + + Assert.assertEquals(tupleDomain1.intersect(tupleDomain2), expectedTupleDomain); + } + + @Test + public void testColumnWiseUnion() + throws Exception + { + TupleDomain tupleDomain1 = TupleDomain.withColumnDomains( + ImmutableMap.builder() + .put(A, Domain.all(String.class)) + .put(B, Domain.notNull(Double.class)) + .put(C, Domain.onlyNull(Long.class)) + .put(D, Domain.singleValue(1L)) + .put(E, Domain.create(SortedRangeSet.of(Range.greaterThanOrEqual(0.0)), true)) + .build()); + + TupleDomain tupleDomain2 = TupleDomain.withColumnDomains( + ImmutableMap.builder() + .put(A, Domain.singleValue("value")) + .put(B, Domain.singleValue(0.0)) + .put(C, Domain.notNull(Long.class)) + .put(D, Domain.singleValue(1L)) + .put(E, Domain.create(SortedRangeSet.of(Range.lessThan(10.0)), false)) + .build()); + + TupleDomain expectedTupleDomain = TupleDomain.withColumnDomains( + ImmutableMap.builder() + .put(A, Domain.all(String.class)) + .put(B, Domain.notNull(Double.class)) + .put(C, Domain.all(Long.class)) + .put(D, Domain.singleValue(1L)) + .put(E, Domain.all(Double.class)) + .build()); + + Assert.assertEquals(tupleDomain1.columnWiseUnion(tupleDomain2), expectedTupleDomain); + } + + @Test + public void testNoneColumnWiseUnion() + throws Exception + { + Assert.assertEquals(TupleDomain.none().columnWiseUnion(TupleDomain.all()), TupleDomain.all()); + Assert.assertEquals(TupleDomain.all().columnWiseUnion(TupleDomain.none()), TupleDomain.all()); + Assert.assertEquals(TupleDomain.none().columnWiseUnion(TupleDomain.none()), TupleDomain.none()); + Assert.assertEquals(TupleDomain.withColumnDomains( + ImmutableMap.of(A, Domain.onlyNull(Long.class))) + .columnWiseUnion( + TupleDomain.withColumnDomains(ImmutableMap.of(A, Domain.notNull(Long.class)))), + TupleDomain.all()); + } + + @Test + public void testMismatchedColumnWiseUnion() + throws Exception + { + TupleDomain tupleDomain1 = TupleDomain.withColumnDomains( + ImmutableMap.of( + A, Domain.all(Double.class), + B, Domain.singleValue("value"))); + + TupleDomain tupleDomain2 = TupleDomain.withColumnDomains( + ImmutableMap.of( + A, Domain.create(SortedRangeSet.of(Range.greaterThanOrEqual(0.0)), true), + C, Domain.singleValue(1L))); + + TupleDomain expectedTupleDomain = TupleDomain.withColumnDomains(ImmutableMap.of(A, Domain.all(Double.class))); + + Assert.assertEquals(tupleDomain1.columnWiseUnion(tupleDomain2), expectedTupleDomain); + } + + @Test + public void testOverlaps() + throws Exception + { + Assert.assertTrue(overlaps( + ImmutableMap.of(), + ImmutableMap.of())); + + Assert.assertTrue(overlaps( + ImmutableMap.of(), + ImmutableMap.of(A, Domain.singleValue(0L)))); + + Assert.assertFalse(overlaps( + ImmutableMap.of(), + ImmutableMap.of(A, Domain.none(Long.class)))); + + Assert.assertFalse(overlaps( + ImmutableMap.of(A, Domain.none(Long.class)), + ImmutableMap.of(A, Domain.none(Long.class)))); + + Assert.assertTrue(overlaps( + ImmutableMap.of(A, Domain.all(Long.class)), + ImmutableMap.of(A, Domain.all(Long.class)))); + + Assert.assertTrue(overlaps( + ImmutableMap.of(A, Domain.singleValue(1L)), + ImmutableMap.of(B, Domain.singleValue("value")))); + + Assert.assertTrue(overlaps( + ImmutableMap.of(A, Domain.singleValue(1L)), + ImmutableMap.of(A, Domain.all(Long.class)))); + + Assert.assertFalse(overlaps( + ImmutableMap.of(A, Domain.singleValue(1L)), + ImmutableMap.of(A, Domain.singleValue(2L)))); + + Assert.assertFalse(overlaps( + ImmutableMap.of( + A, Domain.singleValue(1L), + B, Domain.singleValue(1L)), + ImmutableMap.of( + A, Domain.singleValue(1L), + B, Domain.singleValue(2L)))); + + Assert.assertTrue(overlaps( + ImmutableMap.of( + A, Domain.singleValue(1L), + B, Domain.all(Long.class)), + ImmutableMap.of( + A, Domain.singleValue(1L), + B, Domain.singleValue(2L)))); + } + + @Test + public void testContains() + throws Exception + { + Assert.assertTrue(contains( + ImmutableMap.of(), + ImmutableMap.of())); + + Assert.assertTrue(contains( + ImmutableMap.of(), + ImmutableMap.of(A, Domain.none(Long.class)))); + + Assert.assertTrue(contains( + ImmutableMap.of(), + ImmutableMap.of(A, Domain.all(Long.class)))); + + Assert.assertTrue(contains( + ImmutableMap.of(), + ImmutableMap.of(A, Domain.singleValue(0.0)))); + + Assert.assertFalse(contains( + ImmutableMap.of(A, Domain.none(Long.class)), + ImmutableMap.of())); + + Assert.assertTrue(contains( + ImmutableMap.of(A, Domain.none(Long.class)), + ImmutableMap.of(A, Domain.none(Long.class)))); + + Assert.assertFalse(contains( + ImmutableMap.of(A, Domain.none(Long.class)), + ImmutableMap.of(A, Domain.all(Long.class)))); + + Assert.assertFalse(contains( + ImmutableMap.of(A, Domain.none(Long.class)), + ImmutableMap.of(A, Domain.singleValue(0L)))); + + Assert.assertTrue(contains( + ImmutableMap.of(A, Domain.all(Long.class)), + ImmutableMap.of())); + + Assert.assertTrue(contains( + ImmutableMap.of(A, Domain.all(Long.class)), + ImmutableMap.of(A, Domain.none(Long.class)))); + + Assert.assertTrue(contains( + ImmutableMap.of(A, Domain.all(Long.class)), + ImmutableMap.of(A, Domain.all(Long.class)))); + + Assert.assertTrue(contains( + ImmutableMap.of(A, Domain.all(Long.class)), + ImmutableMap.of(A, Domain.singleValue(0L)))); + + Assert.assertFalse(contains( + ImmutableMap.of(A, Domain.singleValue(0L)), + ImmutableMap.of())); + + Assert.assertTrue(contains( + ImmutableMap.of(A, Domain.singleValue(0L)), + ImmutableMap.of(A, Domain.none(Long.class)))); + + Assert.assertFalse(contains( + ImmutableMap.of(A, Domain.singleValue(0L)), + ImmutableMap.of(A, Domain.all(Long.class)))); + + Assert.assertTrue(contains( + ImmutableMap.of(A, Domain.singleValue(0L)), + ImmutableMap.of(A, Domain.singleValue(0L)))); + + Assert.assertFalse(contains( + ImmutableMap.of(A, Domain.singleValue(0L)), + ImmutableMap.of(B, Domain.singleValue("value")))); + + Assert.assertFalse(contains( + ImmutableMap.of( + A, Domain.singleValue(0L), + B, Domain.singleValue("value")), + ImmutableMap.of(B, Domain.singleValue("value")))); + + Assert.assertTrue(contains( + ImmutableMap.of( + A, Domain.singleValue(0L), + B, Domain.singleValue("value")), + ImmutableMap.of(B, Domain.none(String.class)))); + + Assert.assertTrue(contains( + ImmutableMap.of( + A, Domain.singleValue(0L), + B, Domain.singleValue("value")), + ImmutableMap.of( + A, Domain.singleValue(1L), + B, Domain.none(String.class)))); + + Assert.assertTrue(contains( + ImmutableMap.of( + B, Domain.singleValue("value")), + ImmutableMap.of( + A, Domain.singleValue(0L), + B, Domain.singleValue("value")))); + + Assert.assertTrue(contains( + ImmutableMap.of( + A, Domain.all(Long.class), + B, Domain.singleValue("value")), + ImmutableMap.of( + A, Domain.singleValue(0L), + B, Domain.singleValue("value")))); + + Assert.assertFalse(contains( + ImmutableMap.of( + A, Domain.all(Long.class), + B, Domain.singleValue("value")), + ImmutableMap.of( + A, Domain.singleValue(0L), + B, Domain.singleValue("value2")))); + + Assert.assertTrue(contains( + ImmutableMap.of( + A, Domain.all(Long.class), + B, Domain.singleValue("value")), + ImmutableMap.of( + A, Domain.singleValue(0L), + B, Domain.singleValue("value2"), + C, Domain.none(String.class)))); + + Assert.assertFalse(contains( + ImmutableMap.of( + A, Domain.all(Long.class), + B, Domain.singleValue("value"), + C, Domain.none(String.class)), + ImmutableMap.of( + A, Domain.singleValue(0L), + B, Domain.singleValue("value2")))); + + Assert.assertTrue(contains( + ImmutableMap.of( + A, Domain.all(Long.class), + B, Domain.singleValue("value"), + C, Domain.none(String.class)), + ImmutableMap.of( + A, Domain.singleValue(0L), + B, Domain.none(String.class)))); + } + + @Test + public void testEquals() + throws Exception + { + Assert.assertTrue(equals( + ImmutableMap.of(), + ImmutableMap.of())); + + Assert.assertTrue(equals( + ImmutableMap.of(), + ImmutableMap.of(A, Domain.all(Long.class)))); + + Assert.assertFalse(equals( + ImmutableMap.of(), + ImmutableMap.of(A, Domain.none(Long.class)))); + + Assert.assertFalse(equals( + ImmutableMap.of(), + ImmutableMap.of(A, Domain.singleValue(0L)))); + + Assert.assertTrue(equals( + ImmutableMap.of(A, Domain.all(Long.class)), + ImmutableMap.of(A, Domain.all(Long.class)))); + + Assert.assertFalse(equals( + ImmutableMap.of(A, Domain.all(Long.class)), + ImmutableMap.of(A, Domain.none(Long.class)))); + + Assert.assertFalse(equals( + ImmutableMap.of(A, Domain.all(Long.class)), + ImmutableMap.of(A, Domain.singleValue(0L)))); + + Assert.assertTrue(equals( + ImmutableMap.of(A, Domain.none(Long.class)), + ImmutableMap.of(A, Domain.none(Long.class)))); + + Assert.assertFalse(equals( + ImmutableMap.of(A, Domain.none(Long.class)), + ImmutableMap.of(A, Domain.singleValue(0L)))); + + Assert.assertTrue(equals( + ImmutableMap.of(A, Domain.singleValue(0L)), + ImmutableMap.of(A, Domain.singleValue(0L)))); + + Assert.assertFalse(equals( + ImmutableMap.of(A, Domain.singleValue(0L)), + ImmutableMap.of(B, Domain.singleValue(0L)))); + + Assert.assertFalse(equals( + ImmutableMap.of(A, Domain.singleValue(0L)), + ImmutableMap.of(A, Domain.singleValue(1L)))); + + Assert.assertTrue(equals( + ImmutableMap.of(A, Domain.all(Long.class)), + ImmutableMap.of(B, Domain.all(String.class)))); + + Assert.assertTrue(equals( + ImmutableMap.of(A, Domain.none(Long.class)), + ImmutableMap.of(B, Domain.none(String.class)))); + + Assert.assertTrue(equals( + ImmutableMap.of(A, Domain.none(Long.class)), + ImmutableMap.of( + A, Domain.singleValue(0L), + B, Domain.none(String.class)))); + + Assert.assertFalse(equals( + ImmutableMap.of( + A, Domain.singleValue(1L)), + ImmutableMap.of( + A, Domain.singleValue(0L), + B, Domain.none(String.class)))); + + Assert.assertTrue(equals( + ImmutableMap.of( + A, Domain.singleValue(1L), + C, Domain.none(Double.class)), + ImmutableMap.of( + A, Domain.singleValue(0L), + B, Domain.none(String.class)))); + + Assert.assertTrue(equals( + ImmutableMap.of( + A, Domain.singleValue(0L), + B, Domain.all(Double.class)), + ImmutableMap.of( + A, Domain.singleValue(0L), + B, Domain.all(Double.class)))); + + Assert.assertTrue(equals( + ImmutableMap.of( + A, Domain.singleValue(0L), + B, Domain.all(String.class)), + ImmutableMap.of( + A, Domain.singleValue(0L), + C, Domain.all(Double.class)))); + + Assert.assertFalse(equals( + ImmutableMap.of( + A, Domain.singleValue(0L), + B, Domain.all(String.class)), + ImmutableMap.of( + A, Domain.singleValue(1L), + C, Domain.all(Double.class)))); + + Assert.assertFalse(equals( + ImmutableMap.of( + A, Domain.singleValue(0L), + B, Domain.all(String.class)), + ImmutableMap.of( + A, Domain.singleValue(0L), + C, Domain.singleValue(0.0)))); + } + + @Test + public void testIsNone() + throws Exception + { + Assert.assertFalse(TupleDomain.withColumnDomains(ImmutableMap.of()).isNone()); + Assert.assertFalse(TupleDomain.withColumnDomains(ImmutableMap.of(A, Domain.singleValue(0L))).isNone()); + Assert.assertTrue(TupleDomain.withColumnDomains(ImmutableMap.of(A, Domain.none(Long.class))).isNone()); + Assert.assertFalse(TupleDomain.withColumnDomains(ImmutableMap.of(A, Domain.all(Long.class))).isNone()); + Assert.assertTrue(TupleDomain.withColumnDomains(ImmutableMap.of(A, Domain.all(Long.class), B, Domain.none(Long.class))).isNone()); + } + + @Test + public void testIsAll() + throws Exception + { + Assert.assertTrue(TupleDomain.withColumnDomains(ImmutableMap.of()).isAll()); + Assert.assertFalse(TupleDomain.withColumnDomains(ImmutableMap.of(A, Domain.singleValue(0L))).isAll()); + Assert.assertTrue(TupleDomain.withColumnDomains(ImmutableMap.of(A, Domain.all(Long.class))).isAll()); + Assert.assertFalse(TupleDomain.withColumnDomains(ImmutableMap.of(A, Domain.singleValue(0L), B, Domain.all(Long.class))).isAll()); + } + + @Test + public void testExtractFixedValues() + throws Exception + { + Assert.assertEquals( + TupleDomain.withColumnDomains( + ImmutableMap.builder() + .put(A, Domain.all(Double.class)) + .put(B, Domain.singleValue("value")) + .put(C, Domain.onlyNull(Long.class)) + .put(D, Domain.create(SortedRangeSet.of(Range.equal(1L)), true)) + .build()) + .extractFixedValues(), + ImmutableMap.>of(B, "value")); + } + + @Test + public void testSingleValuesMapToDomain() + throws Exception + { + Assert.assertEquals( + TupleDomain.withFixedValues( + ImmutableMap.>builder() + .put(A, 1L) + .put(B, "value") + .put(C, 0.01) + .put(D, true) + .build()), + TupleDomain.withColumnDomains(ImmutableMap.builder() + .put(A, Domain.singleValue(1L)) + .put(B, Domain.singleValue("value")) + .put(C, Domain.singleValue(0.01)) + .put(D, Domain.singleValue(true)) + .build())); + } + + @Test + public void testJsonSerialization() + throws Exception + { + final ObjectMapper mapper = new ObjectMapper(); + + // Normally, Presto server takes care of registering plugin types with Jackson... + // But since we know that ColumnHandle will always be a TestingColumnHandle in this test, + // let's just always deserialize ColumnHandle as a TestingColumnHandle. + mapper.registerModule(new SimpleModule().addDeserializer(ColumnHandle.class, new JsonDeserializer() + { + @Override + public ColumnHandle deserialize(JsonParser jsonParser, DeserializationContext deserializationContext) + throws IOException + { + return mapper.readValue(jsonParser, TestingColumnHandle.class); + } + })); + + TupleDomain tupleDomain = TupleDomain.all(); + Assert.assertEquals(tupleDomain, mapper.readValue(mapper.writeValueAsString(tupleDomain), TupleDomain.class)); + + tupleDomain = TupleDomain.none(); + Assert.assertEquals(tupleDomain, mapper.readValue(mapper.writeValueAsString(tupleDomain), TupleDomain.class)); + + tupleDomain = TupleDomain.withFixedValues(ImmutableMap.>of(A, 1L, B, "abc")); + Assert.assertEquals(tupleDomain, mapper.readValue(mapper.writeValueAsString(tupleDomain), TupleDomain.class)); + } + + private boolean overlaps(Map domains1, Map domains2) + { + TupleDomain tupleDomain1 = TupleDomain.withColumnDomains(domains1); + TupleDomain tupleDOmain2 = TupleDomain.withColumnDomains(domains2); + return tupleDomain1.overlaps(tupleDOmain2); + } + + private boolean contains(Map superSet, Map subSet) + { + TupleDomain superSetTupleDomain = TupleDomain.withColumnDomains(superSet); + TupleDomain subSetTupleDomain = TupleDomain.withColumnDomains(subSet); + return superSetTupleDomain.contains(subSetTupleDomain); + } + + private boolean equals(Map domains1, Map domains2) + { + TupleDomain tupleDomain1 = TupleDomain.withColumnDomains(domains1); + TupleDomain tupleDOmain2 = TupleDomain.withColumnDomains(domains2); + return tupleDomain1.equals(tupleDOmain2); + } +} diff --git a/presto-spi/src/test/java/com/facebook/presto/spi/TestingColumnHandle.java b/presto-spi/src/test/java/com/facebook/presto/spi/TestingColumnHandle.java new file mode 100644 index 0000000000000..c514ff4d318f5 --- /dev/null +++ b/presto-spi/src/test/java/com/facebook/presto/spi/TestingColumnHandle.java @@ -0,0 +1,64 @@ +/* + * 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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Objects; +import com.google.common.base.Preconditions; + +public class TestingColumnHandle + implements ColumnHandle +{ + private final String name; + + @JsonCreator + public TestingColumnHandle(@JsonProperty("name") String name) + { + this.name = Preconditions.checkNotNull(name, "name is null"); + } + + @JsonProperty + public String getName() + { + return name; + } + + @Override + public int hashCode() + { + return Objects.hashCode(name); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final TestingColumnHandle other = (TestingColumnHandle) obj; + return Objects.equal(this.name, other.name); + } + + @Override + public String toString() + { + return Objects.toStringHelper(this) + .add("name", name) + .toString(); + } +}