From 3d874b1397825899662e192deada505968e9da07 Mon Sep 17 00:00:00 2001 From: Sayari Mukherjee Date: Thu, 28 Nov 2024 16:15:55 +0530 Subject: [PATCH 01/25] Update lucene-analyzers-common and lucene-queryparser to 8.10.0 Resolves 3 Medium cves. --- pom.xml | 17 ++++++++++------- presto-pinot-toolkit/pom.xml | 4 ++++ 2 files changed, 14 insertions(+), 7 deletions(-) diff --git a/pom.xml b/pom.xml index ee4920c08462..02554a7d6077 100644 --- a/pom.xml +++ b/pom.xml @@ -58,6 +58,7 @@ 2.12.7 1.53 7.5 + 8.10.0 3.8.0 1.2.13 1.13.1 @@ -2038,13 +2039,13 @@ org.apache.lucene lucene-analyzers-common - 7.2.1 - - - org.apache.lucene - lucene-core - - + ${dep.lucene.version} + + + + org.apache.lucene + lucene-queryparser + ${dep.lucene.version} @@ -2344,6 +2345,8 @@ META-INF.versions.9.module-info META-INF.versions.11.module-info + + META-INF.versions.9.org.apache.lucene.* diff --git a/presto-pinot-toolkit/pom.xml b/presto-pinot-toolkit/pom.xml index 332f3b7f6acd..d7a42e000765 100644 --- a/presto-pinot-toolkit/pom.xml +++ b/presto-pinot-toolkit/pom.xml @@ -22,6 +22,10 @@ org.apache.pinot presto-pinot-driver + + org.apache.lucene + lucene-core + commons-io commons-io From 6bce6c2267d1e4f684e8f278e1b16167c49876c9 Mon Sep 17 00:00:00 2001 From: auden-woolfson Date: Mon, 11 Nov 2024 13:19:48 -0800 Subject: [PATCH 02/25] Add invalid character checking and testing for async page transport servlet --- .../server/AsyncPageTransportServlet.java | 15 +++++++++ .../server/TestAsyncPageTransportServlet.java | 33 +++++++++++++++++++ 2 files changed, 48 insertions(+) diff --git a/presto-main/src/main/java/com/facebook/presto/server/AsyncPageTransportServlet.java b/presto-main/src/main/java/com/facebook/presto/server/AsyncPageTransportServlet.java index 7619c3bb6f5d..57eaa2bab363 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/AsyncPageTransportServlet.java +++ b/presto-main/src/main/java/com/facebook/presto/server/AsyncPageTransportServlet.java @@ -43,6 +43,7 @@ import javax.servlet.http.HttpServletResponse; import java.io.IOException; +import java.util.Enumeration; import java.util.List; import java.util.concurrent.Executor; import java.util.concurrent.ScheduledExecutorService; @@ -128,6 +129,20 @@ protected void parseURI(String requestURI, HttpServletRequest request, HttpServl OutputBufferId bufferId = null; long token = 0; + if (request != null) { + Enumeration headerNames = request.getHeaderNames(); + while (headerNames.hasMoreElements()) { + String headerName = headerNames.nextElement(); + String headerValue = request.getHeader(headerName); + if (headerName.contains("\r") || headerName.contains("\n")) { + throw new IllegalArgumentException(format("Invalid header name: %s", headerName)); + } + if (headerValue.contains("\r") || headerValue.contains("\n")) { + throw new IllegalArgumentException(format("Invalid header value: %s", headerValue)); + } + } + } + int previousIndex = -1; for (int part = 0; part < 8; part++) { int nextIndex = requestURI.indexOf('/', previousIndex + 1); diff --git a/presto-tests/src/test/java/com/facebook/presto/server/TestAsyncPageTransportServlet.java b/presto-tests/src/test/java/com/facebook/presto/server/TestAsyncPageTransportServlet.java index d67dd943a747..aa567502bd7f 100644 --- a/presto-tests/src/test/java/com/facebook/presto/server/TestAsyncPageTransportServlet.java +++ b/presto-tests/src/test/java/com/facebook/presto/server/TestAsyncPageTransportServlet.java @@ -15,6 +15,10 @@ import com.facebook.presto.execution.TaskId; import com.facebook.presto.execution.buffer.OutputBuffers.OutputBufferId; +import com.google.common.collect.ImmutableListMultimap; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ListMultimap; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import javax.servlet.http.HttpServletRequest; @@ -23,6 +27,7 @@ import java.io.IOException; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertThrows; import static org.testng.Assert.fail; @Test(singleThreaded = true) @@ -34,6 +39,7 @@ class TestServlet TaskId taskId; OutputBufferId bufferId; String requestURI; + HttpServletRequest request; long token; void parse(String uri) throws IOException @@ -41,6 +47,11 @@ void parse(String uri) throws IOException parseURI(uri, null, null); } + void parse(String uri, HttpServletRequest request) throws IOException + { + parseURI(uri, request, null); + } + @Override protected void processRequest( String requestURI, TaskId taskId, OutputBufferId bufferId, long token, @@ -50,6 +61,7 @@ protected void processRequest( this.taskId = taskId; this.bufferId = bufferId; this.token = token; + this.request = request; } @Override @@ -80,6 +92,27 @@ public void testParsing() assertEquals(789, servlet.token); } + @DataProvider(name = "testSanitizationProvider") + public Object[][] testSanitizationProvider() + { + return new Object[][] { + {"ke\ny", "value"}, + {"key", "valu\ne"}, + {"ke\ry", "value"}, + {"key", "valu\re"}}; + } + + @Test(dataProvider = "testSanitizationProvider") + public void testSanitization(String key, String value) + { + ListMultimap headers = ImmutableListMultimap.of(key, value); + HttpServletRequest request = new MockHttpServletRequest(headers, "", ImmutableMap.of()); + TestServlet servlet = new TestServlet(); + assertThrows( + IllegalArgumentException.class, + () -> { servlet.parse("/v1/task/async/0.1.2.3.4/results/456/789", request); }); + } + @Test (expectedExceptions = { IllegalArgumentException.class }) public void testParseTooFewElements() { From 22a70a7cf3b7e6e86d30d42d0a5d4034c4c2716c Mon Sep 17 00:00:00 2001 From: Zac Blanco Date: Mon, 22 Apr 2024 15:15:16 -0400 Subject: [PATCH 03/25] [Iceberg] Collect superset of supported statistics The set of ColumnStatisticsMetadata defined by the Hive and non-Hive connectors are not equivalent. However, it is possible to collect the superset of the relevant metadata and use it for ANALYZE. The returned statistics just need to be filtered out to contain only the relevant column statistics. This may include duplicate calculations for some statistics. For example, with distinct values Iceberg puffin files can store the result of sketch_theta for distinct values, but the code path for storing the statistic in the HMS requires a direct value from approx_distinct. Thus, ANALYZE may compute a value twice. --- .../presto/hive/HiveStatisticsUtil.java | 25 ++++- .../presto/iceberg/IcebergHiveMetadata.java | 44 +++++++-- .../presto/iceberg/util/StatisticsUtil.java | 1 + .../presto/iceberg/IcebergQueryRunner.java | 2 +- .../hive/TestIcebergHiveStatistics.java | 93 ++++++++++++++++++- .../sql/planner/BasePlanFragmenter.java | 3 +- .../presto/sql/planner/LogicalPlanner.java | 3 +- .../planner/StatisticsAggregationPlanner.java | 7 +- 8 files changed, 162 insertions(+), 16 deletions(-) diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/HiveStatisticsUtil.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/HiveStatisticsUtil.java index f05c524f0012..b138a899c94c 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/HiveStatisticsUtil.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/HiveStatisticsUtil.java @@ -28,11 +28,13 @@ import java.util.Map; import java.util.Optional; import java.util.OptionalLong; +import java.util.Set; import static com.facebook.presto.common.type.BigintType.BIGINT; import static com.facebook.presto.hive.metastore.Statistics.fromComputedStatistics; import static com.facebook.presto.spi.statistics.TableStatisticType.ROW_COUNT; import static com.google.common.base.Verify.verify; +import static com.google.common.collect.ImmutableMap.toImmutableMap; public final class HiveStatisticsUtil { @@ -61,9 +63,14 @@ public static PartitionStatistics createPartitionStatistics( ConnectorSession session, Map columnTypes, ComputedStatistics computedStatistics, + Set supportedColumnStatistics, DateTimeZone timeZone) { - Map computedColumnStatistics = computedStatistics.getColumnStatistics(); + Map computedColumnStatistics = computedStatistics.getColumnStatistics() + .entrySet() + .stream() + .filter((entry) -> supportedColumnStatistics.contains(entry.getKey())) + .collect(toImmutableMap(Map.Entry::getKey, Map.Entry::getValue)); Block rowCountBlock = Optional.ofNullable(computedStatistics.getTableStatistics().get(ROW_COUNT)) .orElseThrow(() -> new VerifyException("rowCount not present")); @@ -73,6 +80,15 @@ public static PartitionStatistics createPartitionStatistics( return createPartitionStatistics(session, rowCountOnlyBasicStatistics, columnTypes, computedColumnStatistics, timeZone); } + public static PartitionStatistics createPartitionStatistics( + ConnectorSession session, + Map columnTypes, + ComputedStatistics computedStatistics, + DateTimeZone timeZone) + { + return createPartitionStatistics(session, columnTypes, computedStatistics, computedStatistics.getColumnStatistics().keySet(), timeZone); + } + public static Map getColumnStatistics(Map, ComputedStatistics> statistics, List partitionValues) { return Optional.ofNullable(statistics.get(partitionValues)) @@ -81,10 +97,11 @@ public static Map getColumnStatistics(Map hiveColumnStatistics = getHiveSupportedColumnStatistics(session, icebergTable, tableMetadata); + Set supportedStatistics = ImmutableSet.builder() + .addAll(hiveColumnStatistics) + // iceberg table-supported statistics + .addAll(super.getStatisticsCollectionMetadata(session, tableMetadata).getColumnStatistics()) + .build(); + Set tableStatistics = ImmutableSet.of(ROW_COUNT); + return new TableStatisticsMetadata(supportedStatistics, tableStatistics, emptyList()); + } + + private Set getHiveSupportedColumnStatistics(ConnectorSession session, org.apache.iceberg.Table table, ConnectorTableMetadata tableMetadata) + { MetricsConfig metricsConfig = MetricsConfig.forTable(table); - Set columnStatistics = tableMetadata.getColumns().stream() - .filter(column -> !column.isHidden() && metricsConfig.columnMode(column.getName()) != None.get()) + return tableMetadata.getColumns().stream() + .filter(column -> !column.isHidden()) + .filter(column -> metricsConfig.columnMode(column.getName()) != None.get()) .flatMap(meta -> { try { return metastore.getSupportedColumnStatistics(getMetastoreContext(session), meta.getType()) @@ -494,9 +507,6 @@ public TableStatisticsMetadata getStatisticsCollectionMetadata(ConnectorSession } }) .collect(toImmutableSet()); - - Set tableStatistics = ImmutableSet.of(ROW_COUNT); - return new TableStatisticsMetadata(columnStatistics, tableStatistics, emptyList()); } @Override @@ -525,11 +535,31 @@ public void finishStatisticsCollection(ConnectorSession session, ConnectorTableH Map, ComputedStatistics> computedStatisticsMap = createComputedStatisticsToPartitionMap(computedStatistics, partitionColumnNames, columnTypes); // commit analyze to unpartitioned table - PartitionStatistics tableStatistics = createPartitionStatistics(session, columnTypes, computedStatisticsMap.get(ImmutableList.of()), timeZone); + ConnectorTableMetadata metadata = getTableMetadata(session, tableHandle); + org.apache.iceberg.Table icebergTable = getIcebergTable(session, icebergTableHandle.getSchemaTableName()); + Set hiveSupportedStatistics = getHiveSupportedColumnStatistics(session, icebergTable, metadata); + PartitionStatistics tableStatistics = createPartitionStatistics( + session, + columnTypes, + computedStatisticsMap.get(ImmutableList.of()), + hiveSupportedStatistics, + timeZone); metastore.updateTableStatistics(metastoreContext, table.getDatabaseName(), table.getTableName(), oldStats -> updatePartitionStatistics(oldStats, tableStatistics)); + + Set icebergSupportedStatistics = super.getStatisticsCollectionMetadata(session, metadata).getColumnStatistics(); + Collection icebergComputedStatistics = computedStatistics.stream().map(stat -> { + ComputedStatistics.Builder builder = ComputedStatistics.builder(stat.getGroupingColumns(), stat.getGroupingValues()); + stat.getTableStatistics() + .forEach(builder::addTableStatistic); + stat.getColumnStatistics().entrySet().stream() + .filter(entry -> icebergSupportedStatistics.contains(entry.getKey())) + .forEach(entry -> builder.addColumnStatistic(entry.getKey(), entry.getValue())); + return builder.build(); + }).collect(toImmutableList()); + super.finishStatisticsCollection(session, tableHandle, icebergComputedStatistics); } @Override diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/util/StatisticsUtil.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/util/StatisticsUtil.java index a0587ccd8b06..f19ae11b0795 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/util/StatisticsUtil.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/util/StatisticsUtil.java @@ -103,6 +103,7 @@ public static TableStatistics mergeHiveStatistics(TableStatistics icebergStatist .setRange(icebergColumnStats.getRange()) .setNullsFraction(icebergColumnStats.getNullsFraction()) .setDistinctValuesCount(icebergColumnStats.getDistinctValuesCount()) + .setHistogram(icebergColumnStats.getHistogram()) .setRange(icebergColumnStats.getRange()); if (hiveColumnStats != null) { // NDVs diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/IcebergQueryRunner.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/IcebergQueryRunner.java index 11be7a0e9857..0d12c203b80f 100644 --- a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/IcebergQueryRunner.java +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/IcebergQueryRunner.java @@ -215,7 +215,7 @@ public static DistributedQueryRunner createIcebergQueryRunner( queryRunner.createCatalog("jmx", "jmx"); } - if (catalogType == HIVE.name()) { + if (catalogType.equals(HIVE.name())) { ExtendedHiveMetastore metastore = getFileHiveMetastore(icebergDataDirectory); if (!metastore.getDatabase(METASTORE_CONTEXT, "tpch").isPresent()) { queryRunner.execute("CREATE SCHEMA tpch"); diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/hive/TestIcebergHiveStatistics.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/hive/TestIcebergHiveStatistics.java index 98fd3e94c2f4..be90d5fc0dd0 100644 --- a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/hive/TestIcebergHiveStatistics.java +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/hive/TestIcebergHiveStatistics.java @@ -20,11 +20,27 @@ import com.facebook.presto.common.predicate.TupleDomain; import com.facebook.presto.common.predicate.ValueSet; import com.facebook.presto.common.transaction.TransactionId; +import com.facebook.presto.hive.HdfsConfiguration; +import com.facebook.presto.hive.HdfsConfigurationInitializer; +import com.facebook.presto.hive.HdfsEnvironment; +import com.facebook.presto.hive.HiveClientConfig; +import com.facebook.presto.hive.HiveHdfsConfiguration; +import com.facebook.presto.hive.MetastoreClientConfig; +import com.facebook.presto.hive.authentication.NoHdfsAuthentication; +import com.facebook.presto.hive.metastore.ExtendedHiveMetastore; +import com.facebook.presto.hive.metastore.file.FileHiveMetastore; +import com.facebook.presto.iceberg.CatalogType; import com.facebook.presto.iceberg.IcebergColumnHandle; +import com.facebook.presto.iceberg.IcebergHiveTableOperationsConfig; import com.facebook.presto.iceberg.IcebergMetadataColumn; +import com.facebook.presto.iceberg.IcebergUtil; +import com.facebook.presto.metadata.CatalogManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.ConnectorId; import com.facebook.presto.spi.Constraint; +import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.SchemaTableName; import com.facebook.presto.spi.TableHandle; import com.facebook.presto.spi.analyzer.MetadataResolver; import com.facebook.presto.spi.plan.TableScanNode; @@ -39,16 +55,22 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.UpdateStatistics; import org.intellij.lang.annotations.Language; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; +import java.io.File; +import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; @@ -56,9 +78,14 @@ import static com.facebook.presto.common.type.DoubleType.DOUBLE; import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.PARTITION_KEY; import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.metastore.InMemoryCachingHiveMetastore.memoizeMetastore; +import static com.facebook.presto.iceberg.CatalogType.HIVE; +import static com.facebook.presto.iceberg.IcebergQueryRunner.ICEBERG_CATALOG; +import static com.facebook.presto.iceberg.IcebergQueryRunner.TEST_DATA_DIRECTORY; import static com.facebook.presto.iceberg.IcebergQueryRunner.createIcebergQueryRunner; import static com.facebook.presto.iceberg.IcebergSessionProperties.HIVE_METASTORE_STATISTICS_MERGE_STRATEGY; import static com.facebook.presto.iceberg.IcebergSessionProperties.PUSHDOWN_FILTER_ENABLED; +import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED; import static com.facebook.presto.spi.statistics.ColumnStatisticType.NUMBER_OF_DISTINCT_VALUES; import static com.facebook.presto.spi.statistics.ColumnStatisticType.TOTAL_SIZE_IN_BYTES; import static com.facebook.presto.testing.assertions.Assert.assertEquals; @@ -159,6 +186,7 @@ public void testStatsWithPartitionedTableAnalyzed() assertQuerySucceeds("CREATE TABLE statsWithPartitionAnalyze WITH (partitioning = ARRAY['orderdate']) as SELECT * FROM statsNoPartitionAnalyze"); assertQuerySucceeds("ANALYZE statsNoPartitionAnalyze"); assertQuerySucceeds("ANALYZE statsWithPartitionAnalyze"); + deleteTableStatistics("statsWithPartitionAnalyze"); Metadata meta = getQueryRunner().getMetadata(); TransactionId txid = getQueryRunner().getTransactionManager().beginTransaction(false); Session session = getSession().beginTransactionId(txid, getQueryRunner().getTransactionManager(), new AllowAllAccessControl()); @@ -295,12 +323,17 @@ public void testHiveStatisticsMergeFlags() { assertQuerySucceeds("CREATE TABLE mergeFlagsStats (i int, v varchar)"); assertQuerySucceeds("INSERT INTO mergeFlagsStats VALUES (0, '1'), (1, '22'), (2, '333'), (NULL, 'aaaaa'), (4, NULL)"); - assertQuerySucceeds("ANALYZE mergeFlagsStats"); // stats stored in + assertQuerySucceeds("ANALYZE mergeFlagsStats"); + + // invalidate puffin files so only hive stats can be returned + deleteTableStatistics("mergeFlagsStats"); + // Test stats without merging doesn't return NDVs or data size Session session = Session.builder(getSession()) .setCatalogSessionProperty("iceberg", HIVE_METASTORE_STATISTICS_MERGE_STRATEGY, "") .build(); TableStatistics stats = getTableStatistics(session, "mergeFlagsStats"); + Map columnStatistics = getColumnNameMap(stats); assertEquals(columnStatistics.get("i").getDistinctValuesCount(), Estimate.unknown()); assertEquals(columnStatistics.get("i").getDataSize(), Estimate.unknown()); @@ -468,6 +501,64 @@ static void assertStatValue(StatsSchema column, MaterializedResult result, Set new RuntimeException("Catalog directory does not exist: " + getCatalogDirectory(HIVE))), + "test"); + return memoizeMetastore(fileHiveMetastore, false, 1000, 0); + } + + protected static HdfsEnvironment getHdfsEnvironment() + { + HiveClientConfig hiveClientConfig = new HiveClientConfig(); + MetastoreClientConfig metastoreClientConfig = new MetastoreClientConfig(); + HdfsConfiguration hdfsConfiguration = new HiveHdfsConfiguration(new HdfsConfigurationInitializer(hiveClientConfig, metastoreClientConfig), + ImmutableSet.of(), + hiveClientConfig); + return new HdfsEnvironment(hdfsConfiguration, metastoreClientConfig, new NoHdfsAuthentication()); + } + + protected File getCatalogDirectory(CatalogType catalogType) + { + Path dataDirectory = getDistributedQueryRunner().getCoordinator().getDataDirectory(); + switch (catalogType) { + case HIVE: + return dataDirectory + .resolve(TEST_DATA_DIRECTORY) + .resolve(HIVE.name()) + .toFile(); + case HADOOP: + case NESSIE: + return dataDirectory.toFile(); + } + + throw new PrestoException(NOT_SUPPORTED, "Unsupported Presto Iceberg catalog type " + catalogType); + } + private static Map getColumnNameMap(TableStatistics statistics) { return statistics.getColumnStatistics().entrySet().stream().collect(Collectors.toMap(e -> diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/BasePlanFragmenter.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/BasePlanFragmenter.java index 190271f3e39a..ceb0186ffb41 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/BasePlanFragmenter.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/BasePlanFragmenter.java @@ -62,6 +62,7 @@ import static com.facebook.presto.SystemSessionProperties.isForceSingleNodeOutput; import static com.facebook.presto.SystemSessionProperties.isSingleNodeExecutionEnabled; +import static com.facebook.presto.SystemSessionProperties.shouldOptimizerUseHistograms; import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED; import static com.facebook.presto.sql.TemporaryTableUtil.assignPartitioningVariables; import static com.facebook.presto.sql.TemporaryTableUtil.assignTemporaryTableColumnNames; @@ -121,7 +122,7 @@ public BasePlanFragmenter( this.idAllocator = requireNonNull(idAllocator, "idAllocator is null"); this.variableAllocator = requireNonNull(variableAllocator, "variableAllocator is null"); this.outputTableWriterNodeIds = ImmutableSet.copyOf(requireNonNull(outputTableWriterNodeIds, "outputTableWriterNodeIds is null")); - this.statisticsAggregationPlanner = new StatisticsAggregationPlanner(variableAllocator, metadata.getFunctionAndTypeManager(), session); + this.statisticsAggregationPlanner = new StatisticsAggregationPlanner(variableAllocator, metadata.getFunctionAndTypeManager(), session, shouldOptimizerUseHistograms(session)); } public SubPlan buildRootFragment(PlanNode root, FragmentProperties properties) 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 9d0d1aaaf180..2b68e642dec9 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 @@ -86,6 +86,7 @@ import java.util.Optional; import java.util.Set; +import static com.facebook.presto.SystemSessionProperties.shouldOptimizerUseHistograms; import static com.facebook.presto.common.type.BigintType.BIGINT; import static com.facebook.presto.common.type.VarbinaryType.VARBINARY; import static com.facebook.presto.metadata.MetadataUtil.getConnectorIdOrThrow; @@ -134,7 +135,7 @@ public LogicalPlanner( this.idAllocator = requireNonNull(idAllocator, "idAllocator is null"); this.metadata = requireNonNull(metadata, "metadata is null"); this.variableAllocator = requireNonNull(variableAllocator, "variableAllocator is null"); - this.statisticsAggregationPlanner = new StatisticsAggregationPlanner(this.variableAllocator, metadata.getFunctionAndTypeManager(), session); + this.statisticsAggregationPlanner = new StatisticsAggregationPlanner(this.variableAllocator, metadata.getFunctionAndTypeManager(), session, shouldOptimizerUseHistograms(session)); this.sqlParser = requireNonNull(sqlParser, "sqlParser is null"); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/StatisticsAggregationPlanner.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/StatisticsAggregationPlanner.java index b84a5e23d7b5..01da076c6807 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/StatisticsAggregationPlanner.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/StatisticsAggregationPlanner.java @@ -58,15 +58,17 @@ public class StatisticsAggregationPlanner { private final VariableAllocator variableAllocator; private final FunctionAndTypeResolver functionAndTypeResolver; + private final boolean useHistograms; private final Session session; private final FunctionAndTypeManager functionAndTypeManager; - public StatisticsAggregationPlanner(VariableAllocator variableAllocator, FunctionAndTypeManager functionAndTypeManager, Session session) + public StatisticsAggregationPlanner(VariableAllocator variableAllocator, FunctionAndTypeManager functionAndTypeManager, Session session, boolean useHistograms) { this.variableAllocator = requireNonNull(variableAllocator, "variableAllocator is null"); this.session = requireNonNull(session, "session is null"); this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionAndTypeManager is null"); this.functionAndTypeResolver = functionAndTypeManager.getFunctionAndTypeResolver(); + this.useHistograms = useHistograms; } public TableStatisticAggregation createStatisticsAggregation(TableStatisticsMetadata statisticsMetadata, Map columnToVariableMap) @@ -105,6 +107,9 @@ public TableStatisticAggregation createStatisticsAggregation(TableStatisticsMeta } for (ColumnStatisticMetadata columnStatisticMetadata : statisticsMetadata.getColumnStatistics()) { + if (!useHistograms && columnStatisticMetadata.getStatisticType() == ColumnStatisticType.HISTOGRAM) { + continue; + } String columnName = columnStatisticMetadata.getColumnName(); ColumnStatisticType statisticType = columnStatisticMetadata.getStatisticType(); VariableReferenceExpression inputVariable = columnToVariableMap.get(columnName); From c4afd0b7e53899ded94fa1fbfe06c9e4c6e2a5c7 Mon Sep 17 00:00:00 2001 From: Zac Blanco Date: Thu, 28 Mar 2024 21:27:31 -0700 Subject: [PATCH 04/25] [Iceberg] Add histogram statistic support Utilizes the sketch_kll function to generate histograms and store them into the Iceberg table's puffin files for table-level statistic storage. Histograms are always collected by ANALYZE, but they are not used by the cost calculator unless enabled via optimizer.use-histograms --- .../com/facebook/presto/common/Utils.java | 70 +++++- .../presto/common/predicate/Marker.java | 5 + .../presto/common/predicate/Range.java | 10 + .../common/predicate/SortedRangeSet.java | 22 ++ .../com/facebook/presto/common/TestUtils.java | 148 ++++++++++++ .../presto/common/predicate/TestMarker.java | 10 + .../common/predicate/TestSortedRangeSet.java | 62 +++++ presto-iceberg/pom.xml | 9 +- .../iceberg/IcebergAbstractMetadata.java | 4 +- .../presto/iceberg/IcebergConfig.java | 17 ++ .../iceberg/IcebergSessionProperties.java | 10 + .../presto/iceberg/TableStatisticsMaker.java | 115 +++++++-- .../iceberg/statistics/KllHistogram.java | 210 +++++++++++++++++ .../iceberg/IcebergDistributedTestBase.java | 220 ++++++++++++++++++ .../presto/iceberg/TestIcebergConfig.java | 7 +- .../iceberg/statistics/TestKllHistogram.java | 166 +++++++++++++ .../cost/ComparisonStatsCalculator.java | 7 +- ...ConnectorFilterStatsCalculatorService.java | 5 + .../facebook/presto/cost/JoinStatsRule.java | 6 +- .../cost/PlanNodeStatsEstimateMath.java | 9 +- .../facebook/presto/cost/StatisticRange.java | 28 +-- .../sql/planner/BasePlanFragmenter.java | 3 +- .../presto/sql/planner/LogicalPlanner.java | 3 +- .../planner/StatisticsAggregationPlanner.java | 5 +- .../presto/sql/rewrite/ShowStatsRewrite.java | 5 + .../presto/cost/TestHistogramCalculator.java | 100 -------- .../cost/TestPlanNodeStatsEstimateMath.java | 18 +- .../cost/TestVariableStatsEstimate.java | 4 +- ...ApproximateStatsOutputRowCountMatcher.java | 54 +++++ .../planner/assertions/PlanMatchPattern.java | 6 + presto-spi/pom.xml | 6 + .../spi/statistics/ColumnStatistics.java | 7 + .../DisjointRangeDomainHistogram.java | 139 +++++------ .../spi/statistics}/HistogramCalculator.java | 65 +++--- .../UniformDistributionHistogram.java | 20 +- .../TestDisjointRangeDomainHistogram.java | 84 ++++--- .../presto/spi/statistics}/TestHistogram.java | 3 +- .../statistics/TestHistogramCalculator.java | 101 ++++++++ .../spi/statistics}/TestUniformHistogram.java | 7 +- 39 files changed, 1441 insertions(+), 329 deletions(-) create mode 100644 presto-common/src/test/java/com/facebook/presto/common/TestUtils.java create mode 100644 presto-iceberg/src/main/java/com/facebook/presto/iceberg/statistics/KllHistogram.java create mode 100644 presto-iceberg/src/test/java/com/facebook/presto/iceberg/statistics/TestKllHistogram.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/cost/TestHistogramCalculator.java create mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/ApproximateStatsOutputRowCountMatcher.java rename {presto-main/src/main/java/com/facebook/presto/cost => presto-spi/src/main/java/com/facebook/presto/spi/statistics}/DisjointRangeDomainHistogram.java (74%) rename {presto-main/src/main/java/com/facebook/presto/cost => presto-spi/src/main/java/com/facebook/presto/spi/statistics}/HistogramCalculator.java (68%) rename {presto-main/src/main/java/com/facebook/presto/cost => presto-spi/src/main/java/com/facebook/presto/spi/statistics}/UniformDistributionHistogram.java (85%) rename {presto-main/src/test/java/com/facebook/presto/cost => presto-spi/src/test/java/com/facebook/presto/spi/statistics}/TestDisjointRangeDomainHistogram.java (81%) rename {presto-main/src/test/java/com/facebook/presto/cost => presto-spi/src/test/java/com/facebook/presto/spi/statistics}/TestHistogram.java (97%) create mode 100644 presto-spi/src/test/java/com/facebook/presto/spi/statistics/TestHistogramCalculator.java rename {presto-main/src/test/java/com/facebook/presto/cost => presto-spi/src/test/java/com/facebook/presto/spi/statistics}/TestUniformHistogram.java (93%) diff --git a/presto-common/src/main/java/com/facebook/presto/common/Utils.java b/presto-common/src/main/java/com/facebook/presto/common/Utils.java index a2b88be85bfc..dc31b1778252 100644 --- a/presto-common/src/main/java/com/facebook/presto/common/Utils.java +++ b/presto-common/src/main/java/com/facebook/presto/common/Utils.java @@ -18,8 +18,14 @@ import com.facebook.presto.common.predicate.Primitives; import com.facebook.presto.common.type.Type; +import javax.annotation.Nullable; + +import java.util.function.Supplier; + import static com.facebook.presto.common.type.TypeUtils.readNativeValue; import static com.facebook.presto.common.type.TypeUtils.writeNativeValue; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; public final class Utils { @@ -30,7 +36,7 @@ private Utils() public static Block nativeValueToBlock(Type type, Object object) { if (object != null && !Primitives.wrap(type.getJavaType()).isInstance(object)) { - throw new IllegalArgumentException(String.format("Object '%s' does not match type %s", object, type.getJavaType())); + throw new IllegalArgumentException(format("Object '%s' does not match type %s", object, type.getJavaType())); } BlockBuilder blockBuilder = type.createBlockBuilder(null, 1); writeNativeValue(type, blockBuilder, object); @@ -49,10 +55,68 @@ public static void checkArgument(boolean expression) } } - public static void checkArgument(boolean expression, String errorMessage) + public static void checkArgument(boolean expression, String message, Object... args) { if (!expression) { - throw new IllegalArgumentException(errorMessage); + throw new IllegalArgumentException(format(message, args)); + } + } + + /** + * Returns a supplier which caches the instance retrieved during the first call to {@code get()} + * and returns that value on subsequent calls to {@code get()}. + */ + public static Supplier memoizedSupplier(Supplier delegate) + { + if (delegate instanceof MemoizingSupplier) { + return delegate; + } + return new MemoizingSupplier<>(delegate); + } + + /** + * Vendored from Guava + */ + static class MemoizingSupplier + implements Supplier + { + volatile Supplier delegate; + volatile boolean initialized; + // "value" does not need to be volatile; visibility piggy-backs + // on volatile read of "initialized". + @Nullable T value; + + MemoizingSupplier(Supplier delegate) + { + this.delegate = requireNonNull(delegate); + } + + @Override + public T get() + { + // A 2-field variant of Double Checked Locking. + if (!initialized) { + synchronized (this) { + if (!initialized) { + T t = delegate.get(); + value = t; + initialized = true; + // Release the delegate to GC. + delegate = null; + return t; + } + } + } + return value; + } + + @Override + public String toString() + { + Supplier delegate = this.delegate; + return "Suppliers.memoize(" + + (delegate == null ? "" : delegate) + + ")"; } } diff --git a/presto-common/src/main/java/com/facebook/presto/common/predicate/Marker.java b/presto-common/src/main/java/com/facebook/presto/common/predicate/Marker.java index f20a87065bcf..76a58d147a28 100644 --- a/presto-common/src/main/java/com/facebook/presto/common/predicate/Marker.java +++ b/presto-common/src/main/java/com/facebook/presto/common/predicate/Marker.java @@ -129,6 +129,11 @@ public Object getValue() return Utils.blockToNativeValue(type, valueBlock.get()); } + public Optional getObjectValue() + { + return valueBlock.map(block -> Utils.blockToNativeValue(type, block)); + } + public Object getPrintableValue(SqlFunctionProperties properties) { if (!valueBlock.isPresent()) { diff --git a/presto-common/src/main/java/com/facebook/presto/common/predicate/Range.java b/presto-common/src/main/java/com/facebook/presto/common/predicate/Range.java index d00a3c77df0e..501996cc6aa8 100644 --- a/presto-common/src/main/java/com/facebook/presto/common/predicate/Range.java +++ b/presto-common/src/main/java/com/facebook/presto/common/predicate/Range.java @@ -247,6 +247,16 @@ public boolean equals(Object obj) Objects.equals(this.high, other.high); } + @Override + public String toString() + { + return (low.getBound() == Marker.Bound.EXACTLY ? "[" : "(") + + low.getObjectValue().orElse(Double.NEGATIVE_INFINITY) + + ".." + + high.getObjectValue().orElse(Double.POSITIVE_INFINITY) + + (high.getBound() == Marker.Bound.EXACTLY ? "]" : ")"); + } + private void appendQuotedValue(StringBuilder buffer, Marker marker, SqlFunctionProperties properties) { buffer.append('"'); diff --git a/presto-common/src/main/java/com/facebook/presto/common/predicate/SortedRangeSet.java b/presto-common/src/main/java/com/facebook/presto/common/predicate/SortedRangeSet.java index 5f1988be005d..4af54a8e2a68 100644 --- a/presto-common/src/main/java/com/facebook/presto/common/predicate/SortedRangeSet.java +++ b/presto-common/src/main/java/com/facebook/presto/common/predicate/SortedRangeSet.java @@ -168,6 +168,28 @@ public Object getSingleValue() return lowIndexedRanges.values().iterator().next().getSingleValue(); } + /** + * Build a new {@link SortedRangeSet} that contains ranges which lie within the argument range + * + * @param span the range which the new set should span + * @return a new range set + */ + public SortedRangeSet subRangeSet(Range span) + { + Builder builder = new Builder(type); + + for (Range range : getOrderedRanges()) { + if (span.contains(range)) { + builder.add(range); + } + else if (span.overlaps(range)) { + builder.add(range.intersect(span)); + } + } + + return builder.build(); + } + @Override public boolean containsValue(Object value) { diff --git a/presto-common/src/test/java/com/facebook/presto/common/TestUtils.java b/presto-common/src/test/java/com/facebook/presto/common/TestUtils.java new file mode 100644 index 000000000000..646a44fc919c --- /dev/null +++ b/presto-common/src/test/java/com/facebook/presto/common/TestUtils.java @@ -0,0 +1,148 @@ +/* + * 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.common; + +import org.testng.annotations.Test; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.function.Supplier; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertSame; +import static org.testng.Assert.fail; + +public class TestUtils +{ + @Test + public void testCheckArgumentFailWithMessage() + { + try { + Utils.checkArgument(false, "test %s", "test"); + fail(); + } + catch (IllegalArgumentException e) { + assertEquals(e.getMessage(), "test test"); + } + } + + @Test + public void testCheckArgumentPassWithMessage() + { + try { + Utils.checkArgument(true, "test %s", "test"); + } + catch (IllegalArgumentException e) { + fail(); + } + } + + @Test + public void testMemoizedSupplierThreadSafe() + throws Throwable + { + Function, Supplier> memoizer = + supplier -> Utils.memoizedSupplier(supplier); + testSupplierThreadSafe(memoizer); + } + + /** + * Vendored from Guava + */ + private void testSupplierThreadSafe(Function, Supplier> memoizer) + throws Throwable + { + final AtomicInteger count = new AtomicInteger(0); + final AtomicReference thrown = new AtomicReference<>(null); + final int numThreads = 3; + final Thread[] threads = new Thread[numThreads]; + final long timeout = TimeUnit.SECONDS.toNanos(60); + + final Supplier supplier = + new Supplier() + { + boolean isWaiting(Thread thread) + { + switch (thread.getState()) { + case BLOCKED: + case WAITING: + case TIMED_WAITING: + return true; + default: + return false; + } + } + + int waitingThreads() + { + int waitingThreads = 0; + for (Thread thread : threads) { + if (isWaiting(thread)) { + waitingThreads++; + } + } + return waitingThreads; + } + + @Override + @SuppressWarnings("ThreadPriorityCheck") // doing our best to test for races + public Boolean get() + { + // Check that this method is called exactly once, by the first + // thread to synchronize. + long t0 = System.nanoTime(); + while (waitingThreads() != numThreads - 1) { + if (System.nanoTime() - t0 > timeout) { + thrown.set( + new TimeoutException( + "timed out waiting for other threads to block" + + " synchronizing on supplier")); + break; + } + Thread.yield(); + } + count.getAndIncrement(); + return Boolean.TRUE; + } + }; + + final Supplier memoizedSupplier = memoizer.apply(supplier); + + for (int i = 0; i < numThreads; i++) { + threads[i] = + new Thread() + { + @Override + public void run() + { + assertSame(Boolean.TRUE, memoizedSupplier.get()); + } + }; + } + for (Thread t : threads) { + t.start(); + } + for (Thread t : threads) { + t.join(); + } + + if (thrown.get() != null) { + throw thrown.get(); + } + assertEquals(1, count.get()); + } +} diff --git a/presto-common/src/test/java/com/facebook/presto/common/predicate/TestMarker.java b/presto-common/src/test/java/com/facebook/presto/common/predicate/TestMarker.java index 362fc8f29c97..1eb4085e7222 100644 --- a/presto-common/src/test/java/com/facebook/presto/common/predicate/TestMarker.java +++ b/presto-common/src/test/java/com/facebook/presto/common/predicate/TestMarker.java @@ -191,6 +191,16 @@ public void testCanonicalize() assertDifferentMarker(Marker.upperUnbounded(BIGINT), Marker.lowerUnbounded(BIGINT), true); } + @Test + public void testGetValue() + { + assertTrue(Marker.exactly(BIGINT, 1L).getObjectValue().isPresent()); + assertTrue(Marker.above(BIGINT, 1L).getObjectValue().isPresent()); + assertTrue(Marker.below(BIGINT, 1L).getObjectValue().isPresent()); + assertFalse(Marker.upperUnbounded(BIGINT).getObjectValue().isPresent()); + assertFalse(Marker.lowerUnbounded(BIGINT).getObjectValue().isPresent()); + } + private void assertSameMarker(Marker marker1, Marker marker2, boolean removeConstants) throws Exception { diff --git a/presto-common/src/test/java/com/facebook/presto/common/predicate/TestSortedRangeSet.java b/presto-common/src/test/java/com/facebook/presto/common/predicate/TestSortedRangeSet.java index f754681359a5..087073c432bd 100644 --- a/presto-common/src/test/java/com/facebook/presto/common/predicate/TestSortedRangeSet.java +++ b/presto-common/src/test/java/com/facebook/presto/common/predicate/TestSortedRangeSet.java @@ -26,6 +26,9 @@ import com.google.common.collect.Iterables; import org.testng.annotations.Test; +import java.util.Arrays; +import java.util.stream.Collectors; + import static com.facebook.presto.common.type.BigintType.BIGINT; import static com.facebook.presto.common.type.BooleanType.BOOLEAN; import static com.facebook.presto.common.type.DoubleType.DOUBLE; @@ -500,6 +503,65 @@ public void testCanonicalize() assertDifferentSet(SortedRangeSet.all(BIGINT), SortedRangeSet.all(BOOLEAN), true); } + @Test + public void testSubRangeSet() + { + // test subrange no overlap below and above + assertEquals(SortedRangeSet.of(Range.lessThan(BIGINT, 10L)) + .subRangeSet(Range.greaterThan(BIGINT, 10L)) + .getOrderedRanges() + .size(), + 0); + assertEquals(SortedRangeSet.of(Range.greaterThan(BIGINT, 10L)) + .subRangeSet(Range.lessThan(BIGINT, 10L)) + .getOrderedRanges() + .size(), + 0); + assertEquals(SortedRangeSet.of(Range.greaterThanOrEqual(BIGINT, 10L)) + .subRangeSet(Range.lessThan(BIGINT, 10L)) + .getOrderedRanges() + .size(), + 0); + assertEquals(SortedRangeSet.of(Range.lessThanOrEqual(BIGINT, 10L)) + .subRangeSet(Range.greaterThan(BIGINT, 10L)) + .getOrderedRanges() + .size(), + 0); + + // test with equal bounds + assertEquals(SortedRangeSet.of(Range.lessThanOrEqual(BIGINT, 10L)) + .subRangeSet(Range.greaterThanOrEqual(BIGINT, 10L)) + .getOrderedRanges() + .size(), + 1); + assertEquals(SortedRangeSet.of(Range.greaterThanOrEqual(BIGINT, 10L)) + .subRangeSet(Range.lessThanOrEqual(BIGINT, 10L)) + .getOrderedRanges() + .size(), + 1); + assertEquals(SortedRangeSet.of(Range.lessThanOrEqual(BIGINT, 10L)) + .subRangeSet(Range.greaterThanOrEqual(BIGINT, 10L)) + .getOrderedRanges().get(0), Range.range(BIGINT, 10L, true, 10L, true)); + // two ranges + assertEquals(SortedRangeSet.of(Range.lessThan(BIGINT, -10L), Range.greaterThan(BIGINT, 10L)) + .subRangeSet(Range.range(BIGINT, -20L, true, 20L, true)).getOrderedRanges(), + Arrays.stream(new Range[] { + Range.range(BIGINT, -20L, true, -10L, false), + Range.range(BIGINT, 10L, false, 20L, true)}) + .collect(Collectors.toList())); + // range entirely contained + assertEquals(SortedRangeSet.of( + Range.lessThan(BIGINT, -10L), + Range.greaterThan(BIGINT, 10L), + Range.range(BIGINT, -5L, true, 5L, true)) + .subRangeSet(Range.range(BIGINT, -20L, true, 20L, true)).getOrderedRanges(), + Arrays.stream(new Range[] { + Range.range(BIGINT, -20L, true, -10L, false), + Range.range(BIGINT, -5L, true, 5L, true), + Range.range(BIGINT, 10L, false, 20L, true)}) + .collect(Collectors.toList())); + } + private void assertSameSet(SortedRangeSet set1, SortedRangeSet set2, boolean removeSafeConstants) throws Exception { diff --git a/presto-iceberg/pom.xml b/presto-iceberg/pom.xml index 7372a8956f93..36cbe08db3b0 100644 --- a/presto-iceberg/pom.xml +++ b/presto-iceberg/pom.xml @@ -510,19 +510,16 @@ presto-cache compile - com.facebook.presto presto-main test - com.facebook.presto presto-parser test - com.facebook.presto presto-analyzer @@ -604,7 +601,7 @@ org.apache.iceberg iceberg-core - 1.5.0 + ${dep.iceberg.version} tests test @@ -634,6 +631,10 @@ + + org.apache.commons + commons-math3 + diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergAbstractMetadata.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergAbstractMetadata.java index c98e726b0949..842bf00f8f19 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergAbstractMetadata.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergAbstractMetadata.java @@ -657,7 +657,7 @@ public TableStatisticsMetadata getStatisticsCollectionMetadata(ConnectorSession MetricsConfig metricsConfig = MetricsConfig.forTable(table); Set columnStatistics = tableMetadata.getColumns().stream() .filter(column -> !column.isHidden() && metricsConfig.columnMode(column.getName()) != None.get()) - .flatMap(meta -> getSupportedColumnStatistics(meta.getName(), meta.getType()).stream()) + .flatMap(meta -> getSupportedColumnStatistics(session, meta.getName(), meta.getType()).stream()) .collect(toImmutableSet()); Set tableStatistics = ImmutableSet.of(ROW_COUNT); @@ -675,7 +675,7 @@ public void finishStatisticsCollection(ConnectorSession session, ConnectorTableH { IcebergTableHandle icebergTableHandle = (IcebergTableHandle) tableHandle; Table icebergTable = getIcebergTable(session, icebergTableHandle.getSchemaTableName()); - TableStatisticsMaker.writeTableStatistics(nodeVersion, icebergTableHandle, icebergTable, session, computedStatistics); + TableStatisticsMaker.writeTableStatistics(nodeVersion, typeManager, icebergTableHandle, icebergTable, session, computedStatistics); } public void rollback() diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergConfig.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergConfig.java index fe11f4b2fefb..42328ddb4e7f 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergConfig.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergConfig.java @@ -24,6 +24,7 @@ import javax.validation.constraints.DecimalMax; import javax.validation.constraints.DecimalMin; +import javax.validation.constraints.Max; import javax.validation.constraints.Min; import javax.validation.constraints.NotNull; @@ -62,6 +63,7 @@ public class IcebergConfig private int metadataPreviousVersionsMax = METADATA_PREVIOUS_VERSIONS_MAX_DEFAULT; private boolean metadataDeleteAfterCommit = METADATA_DELETE_AFTER_COMMIT_ENABLED_DEFAULT; private int metricsMaxInferredColumn = METRICS_MAX_INFERRED_COLUMN_DEFAULTS_DEFAULT; + private int statisticsKllSketchKParameter = 1024; private EnumSet hiveStatisticsMergeFlags = EnumSet.noneOf(ColumnStatisticType.class); private String fileIOImpl = HadoopFileIO.class.getName(); @@ -412,4 +414,19 @@ public IcebergConfig setMaxStatisticsFileCacheSize(DataSize maxStatisticsFileCac this.maxStatisticsFileCacheSize = maxStatisticsFileCacheSize; return this; } + + public int getStatisticsKllSketchKParameter() + { + return this.statisticsKllSketchKParameter; + } + + @Config("iceberg.statistics-kll-sketch-k-parameter") + @Min(8) + @Max(65535) + @ConfigDescription("K parameter for KLL sketches when generating histogram statistics") + public IcebergConfig setStatisticsKllSketchKParameter(int kllSketchKParameter) + { + this.statisticsKllSketchKParameter = kllSketchKParameter; + return this; + } } diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergSessionProperties.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergSessionProperties.java index 5a597d97051b..57f954801f2f 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergSessionProperties.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergSessionProperties.java @@ -64,6 +64,7 @@ public final class IcebergSessionProperties public static final String HIVE_METASTORE_STATISTICS_MERGE_STRATEGY = "hive_statistics_merge_strategy"; public static final String STATISTIC_SNAPSHOT_RECORD_DIFFERENCE_WEIGHT = "statistic_snapshot_record_difference_weight"; public static final String ROWS_FOR_METADATA_OPTIMIZATION_THRESHOLD = "rows_for_metadata_optimization_threshold"; + public static final String STATISTICS_KLL_SKETCH_K_PARAMETER = "statistics_kll_sketch_k_parameter"; private final List> sessionProperties; @@ -184,6 +185,10 @@ public IcebergSessionProperties( "of an Iceberg table exceeds this threshold, metadata optimization would be skipped for " + "the table. A value of 0 means skip metadata optimization directly.", icebergConfig.getRowsForMetadataOptimizationThreshold(), + false)) + .add(integerProperty(STATISTICS_KLL_SKETCH_K_PARAMETER, + "The K parameter for the Apache DataSketches KLL sketch when computing histogram statistics", + icebergConfig.getStatisticsKllSketchKParameter(), false)); nessieConfig.ifPresent((config) -> propertiesBuilder @@ -313,4 +318,9 @@ public static String getNessieReferenceHash(ConnectorSession session) { return session.getProperty(NESSIE_REFERENCE_HASH, String.class); } + + public static int getStatisticsKllSketchKParameter(ConnectorSession session) + { + return session.getProperty(STATISTICS_KLL_SKETCH_K_PARAMETER, Integer.class); + } } diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/TableStatisticsMaker.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/TableStatisticsMaker.java index 238eb09095f2..153c511be0f6 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/TableStatisticsMaker.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/TableStatisticsMaker.java @@ -16,10 +16,14 @@ import com.facebook.airlift.log.Logger; import com.facebook.presto.common.RuntimeUnit; import com.facebook.presto.common.block.Block; +import com.facebook.presto.common.predicate.Range; import com.facebook.presto.common.predicate.TupleDomain; +import com.facebook.presto.common.type.DecimalType; import com.facebook.presto.common.type.FixedWidthType; +import com.facebook.presto.common.type.KllSketchType; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.hive.NodeVersion; +import com.facebook.presto.iceberg.statistics.KllHistogram; import com.facebook.presto.iceberg.statistics.StatisticsFileCache; import com.facebook.presto.iceberg.statistics.StatisticsFileCacheKey; import com.facebook.presto.spi.ConnectorSession; @@ -29,12 +33,15 @@ import com.facebook.presto.spi.statistics.ColumnStatisticType; import com.facebook.presto.spi.statistics.ColumnStatistics; import com.facebook.presto.spi.statistics.ComputedStatistics; +import com.facebook.presto.spi.statistics.DisjointRangeDomainHistogram; import com.facebook.presto.spi.statistics.DoubleRange; import com.facebook.presto.spi.statistics.Estimate; import com.facebook.presto.spi.statistics.TableStatistics; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; import org.apache.datasketches.memory.Memory; import org.apache.datasketches.theta.CompactSketch; import org.apache.iceberg.ContentFile; @@ -62,6 +69,8 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.Pair; +import javax.annotation.Nullable; + import java.io.IOException; import java.io.UncheckedIOException; import java.nio.ByteBuffer; @@ -80,6 +89,7 @@ import static com.facebook.presto.common.type.BigintType.BIGINT; import static com.facebook.presto.common.type.DateType.DATE; +import static com.facebook.presto.common.type.DoubleType.DOUBLE; import static com.facebook.presto.common.type.TimestampType.TIMESTAMP; import static com.facebook.presto.common.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; import static com.facebook.presto.common.type.TypeUtils.isNumericType; @@ -89,10 +99,14 @@ import static com.facebook.presto.iceberg.IcebergErrorCode.ICEBERG_FILESYSTEM_ERROR; import static com.facebook.presto.iceberg.IcebergErrorCode.ICEBERG_INVALID_METADATA; import static com.facebook.presto.iceberg.IcebergSessionProperties.getStatisticSnapshotRecordDifferenceWeight; +import static com.facebook.presto.iceberg.IcebergSessionProperties.getStatisticsKllSketchKParameter; import static com.facebook.presto.iceberg.IcebergUtil.getIdentityPartitions; import static com.facebook.presto.iceberg.Partition.toMap; +import static com.facebook.presto.iceberg.TypeConverter.toPrestoType; +import static com.facebook.presto.iceberg.statistics.KllHistogram.isKllHistogramSupportedType; import static com.facebook.presto.iceberg.util.StatisticsUtil.calculateAndSetTableSize; import static com.facebook.presto.iceberg.util.StatisticsUtil.formatIdentifier; +import static com.facebook.presto.spi.statistics.ColumnStatisticType.HISTOGRAM; import static com.facebook.presto.spi.statistics.ColumnStatisticType.NUMBER_OF_DISTINCT_VALUES; import static com.facebook.presto.spi.statistics.ColumnStatisticType.TOTAL_SIZE_IN_BYTES; import static com.facebook.presto.spi.statistics.SourceInfo.ConfidenceLevel.HIGH; @@ -100,9 +114,11 @@ import static com.google.common.collect.ImmutableMap.toImmutableMap; import static com.google.common.collect.ImmutableSet.toImmutableSet; import static com.google.common.collect.Iterables.getOnlyElement; +import static com.google.common.collect.Iterators.getOnlyElement; import static java.lang.Long.parseLong; import static java.lang.Math.abs; import static java.lang.String.format; +import static java.util.Objects.requireNonNull; import static java.util.UUID.randomUUID; import static java.util.stream.Collectors.toSet; import static org.apache.iceberg.SnapshotSummary.TOTAL_RECORDS_PROP; @@ -112,29 +128,34 @@ public class TableStatisticsMaker private static final Logger log = Logger.get(TableStatisticsMaker.class); private static final String ICEBERG_THETA_SKETCH_BLOB_TYPE_ID = "apache-datasketches-theta-v1"; private static final String ICEBERG_DATA_SIZE_BLOB_TYPE_ID = "presto-sum-data-size-bytes-v1"; + private static final String ICEBERG_KLL_SKETCH_BLOB_TYPE_ID = "presto-kll-sketch-bytes-v1"; private static final String ICEBERG_THETA_SKETCH_BLOB_PROPERTY_NDV_KEY = "ndv"; private static final String ICEBERG_DATA_SIZE_BLOB_PROPERTY_KEY = "data_size"; private final Table icebergTable; private final ConnectorSession session; + private final TypeManager typeManager; private static final String STATISITCS_CACHE_METRIC_FILE_SIZE_FORMAT = "StatisticsFileCache/PuffinFileSize/%s/%s"; private static final String STATISITCS_CACHE_METRIC_FILE_COLUMN_COUNT_FORMAT = "StatisticsFileCache/ColumnCount/%s/%s"; private static final String STATISITCS_CACHE_METRIC_PARTIAL_MISS_FORMAT = "StatisticsFileCache/PartialMiss/%s/%s"; - private TableStatisticsMaker(Table icebergTable, ConnectorSession session) + private TableStatisticsMaker(Table icebergTable, ConnectorSession session, TypeManager typeManager) { - this.icebergTable = icebergTable; - this.session = session; + this.icebergTable = requireNonNull(icebergTable, "icebergTable is null"); + this.session = requireNonNull(session, "session is null"); + this.typeManager = requireNonNull(typeManager, "typeManager is null"); } private static final Map puffinStatWriters = ImmutableMap.builder() .put(NUMBER_OF_DISTINCT_VALUES, TableStatisticsMaker::generateNDVBlob) .put(TOTAL_SIZE_IN_BYTES, TableStatisticsMaker::generateStatSizeBlob) + .put(HISTOGRAM, TableStatisticsMaker::generateKllSketchBlob) .build(); private static final Map puffinStatReaders = ImmutableMap.builder() .put(ICEBERG_THETA_SKETCH_BLOB_TYPE_ID, TableStatisticsMaker::readNDVBlob) .put(ICEBERG_DATA_SIZE_BLOB_TYPE_ID, TableStatisticsMaker::readDataSizeBlob) + .put(ICEBERG_KLL_SKETCH_BLOB_TYPE_ID, TableStatisticsMaker::readKllSketchBlob) .build(); public static TableStatistics getTableStatistics( @@ -147,7 +168,7 @@ public static TableStatistics getTableStatistics( Table icebergTable, List columns) { - return new TableStatisticsMaker(icebergTable, session).makeTableStatistics(statisticsFileCache, tableHandle, currentPredicate, constraint, columns); + return new TableStatisticsMaker(icebergTable, session, typeManager).makeTableStatistics(statisticsFileCache, tableHandle, currentPredicate, constraint, columns); } private TableStatistics makeTableStatistics(StatisticsFileCache statisticsFileCache, @@ -235,7 +256,18 @@ private TableStatistics makeTableStatistics(StatisticsFileCache statisticsFileCa Object min = summary.getMinValues().get(fieldId); Object max = summary.getMaxValues().get(fieldId); if (min instanceof Number && max instanceof Number) { - columnBuilder.setRange(Optional.of(new DoubleRange(((Number) min).doubleValue(), ((Number) max).doubleValue()))); + DoubleRange range = new DoubleRange(((Number) min).doubleValue(), ((Number) max).doubleValue()); + columnBuilder.setRange(Optional.of(range)); + + // the histogram is generated by scanning the entire dataset. It is possible that + // the constraint prevents scanning portions of the table. Given that we know the + // range that the scan provides for a particular column, bound the histogram to the + // scanned range. + + final DoubleRange histRange = range; + columnBuilder.setHistogram(columnBuilder.getHistogram() + .map(histogram -> DisjointRangeDomainHistogram + .addConjunction(histogram, Range.range(DOUBLE, histRange.getMin(), true, histRange.getMax(), true)))); } result.setColumnStatistics(columnHandle, columnBuilder.build()); } @@ -310,12 +342,12 @@ private Partition getSummaryFromFiles(CloseableIterable> files, return summary; } - public static void writeTableStatistics(NodeVersion nodeVersion, IcebergTableHandle tableHandle, Table icebergTable, ConnectorSession session, Collection computedStatistics) + public static void writeTableStatistics(NodeVersion nodeVersion, TypeManager typeManager, IcebergTableHandle tableHandle, Table icebergTable, ConnectorSession session, Collection computedStatistics) { - new TableStatisticsMaker(icebergTable, session).writeTableStatistics(nodeVersion, tableHandle, computedStatistics); + new TableStatisticsMaker(icebergTable, session, typeManager).writeTableStatistics(nodeVersion, typeManager, tableHandle, computedStatistics); } - private void writeTableStatistics(NodeVersion nodeVersion, IcebergTableHandle tableHandle, Collection computedStatistics) + private void writeTableStatistics(NodeVersion nodeVersion, TypeManager typeManager, IcebergTableHandle tableHandle, Collection computedStatistics) { Snapshot snapshot = tableHandle.getIcebergTableName().getSnapshotId().map(icebergTable::snapshot).orElseGet(icebergTable::currentSnapshot); if (snapshot == null) { @@ -335,9 +367,8 @@ private void writeTableStatistics(NodeVersion nodeVersion, IcebergTableHandle ta .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)) .forEach((key, value) -> { Optional.ofNullable(puffinStatWriters.get(key.getStatisticType())) - .ifPresent(generator -> { - writer.add(generator.generate(key, value, icebergTable, snapshot)); - }); + .flatMap(generator -> Optional.ofNullable(generator.generate(key, value, icebergTable, snapshot, typeManager))) + .ifPresent(writer::add); }); writer.finish(); icebergTable.updateStatistics().setStatistics( @@ -362,7 +393,8 @@ private void writeTableStatistics(NodeVersion nodeVersion, IcebergTableHandle ta @FunctionalInterface private interface PuffinBlobGenerator { - Blob generate(ColumnStatisticMetadata metadata, Block value, Table icebergTable, Snapshot snapshot); + @Nullable + Blob generate(ColumnStatisticMetadata metadata, Block value, Table icebergTable, Snapshot snapshot, TypeManager typeManager); } @FunctionalInterface @@ -371,12 +403,12 @@ private interface PuffinBlobReader /** * Reads the stats from the blob and then updates the stats builder argument. */ - void read(BlobMetadata metadata, ByteBuffer blob, ColumnStatistics.Builder stats); + void read(BlobMetadata metadata, ByteBuffer blob, ColumnStatistics.Builder stats, Table icebergTable, TypeManager typeManager); } - private static Blob generateNDVBlob(ColumnStatisticMetadata metadata, Block value, Table icebergTable, Snapshot snapshot) + private static Blob generateNDVBlob(ColumnStatisticMetadata metadata, Block value, Table icebergTable, Snapshot snapshot, TypeManager typeManager) { - int id = getFieldId(metadata, icebergTable); + int id = getField(metadata, icebergTable, snapshot).fieldId(); ByteBuffer raw = VARBINARY.getSlice(value, 0).toByteBuffer(); CompactSketch sketch = CompactSketch.wrap(Memory.wrap(raw, ByteOrder.nativeOrder())); return new Blob( @@ -389,9 +421,9 @@ private static Blob generateNDVBlob(ColumnStatisticMetadata metadata, Block valu ImmutableMap.of(ICEBERG_THETA_SKETCH_BLOB_PROPERTY_NDV_KEY, Long.toString((long) sketch.getEstimate()))); } - private static Blob generateStatSizeBlob(ColumnStatisticMetadata metadata, Block value, Table icebergTable, Snapshot snapshot) + private static Blob generateStatSizeBlob(ColumnStatisticMetadata metadata, Block value, Table icebergTable, Snapshot snapshot, TypeManager typeManager) { - int id = getFieldId(metadata, icebergTable); + int id = getField(metadata, icebergTable, snapshot).fieldId(); long size = BIGINT.getLong(value, 0); return new Blob( ICEBERG_DATA_SIZE_BLOB_TYPE_ID, @@ -403,7 +435,26 @@ private static Blob generateStatSizeBlob(ColumnStatisticMetadata metadata, Block ImmutableMap.of(ICEBERG_DATA_SIZE_BLOB_PROPERTY_KEY, Long.toString(size))); } - private static void readNDVBlob(BlobMetadata metadata, ByteBuffer blob, ColumnStatistics.Builder statistics) + private static Blob generateKllSketchBlob(ColumnStatisticMetadata metadata, Block value, Table icebergTable, Snapshot snapshot, TypeManager typeManager) + { + Types.NestedField field = getField(metadata, icebergTable, snapshot); + KllSketchType sketchType = new KllSketchType(toPrestoType(field.type(), typeManager)); + Slice sketchSlice = sketchType.getSlice(value, 0); + if (value.isNull(0)) { + // this can occur when all inputs to the sketch are null + return null; + } + return new Blob( + ICEBERG_KLL_SKETCH_BLOB_TYPE_ID, + ImmutableList.of(field.fieldId()), + snapshot.snapshotId(), + snapshot.sequenceNumber(), + sketchSlice.toByteBuffer(), + null, + ImmutableMap.of()); + } + + private static void readNDVBlob(BlobMetadata metadata, ByteBuffer blob, ColumnStatistics.Builder statistics, Table icebergTable, TypeManager typeManager) { Optional.ofNullable(metadata.properties().get(ICEBERG_THETA_SKETCH_BLOB_PROPERTY_NDV_KEY)) .ifPresent(ndvProp -> { @@ -418,7 +469,7 @@ private static void readNDVBlob(BlobMetadata metadata, ByteBuffer blob, ColumnSt }); } - private static void readDataSizeBlob(BlobMetadata metadata, ByteBuffer blob, ColumnStatistics.Builder statistics) + private static void readDataSizeBlob(BlobMetadata metadata, ByteBuffer blob, ColumnStatistics.Builder statistics, Table icebergTable, TypeManager typeManager) { Optional.ofNullable(metadata.properties().get(ICEBERG_DATA_SIZE_BLOB_PROPERTY_KEY)) .ifPresent(sizeProp -> { @@ -433,9 +484,17 @@ private static void readDataSizeBlob(BlobMetadata metadata, ByteBuffer blob, Col }); } - private static int getFieldId(ColumnStatisticMetadata metadata, Table icebergTable) + private static void readKllSketchBlob(BlobMetadata metadata, ByteBuffer blob, ColumnStatistics.Builder statistics, Table icebergTable, TypeManager typeManager) + { + statistics.setHistogram(Optional.ofNullable(icebergTable.schemas().get(icebergTable.snapshot(metadata.snapshotId()).schemaId())) + .map(schema -> toPrestoType(schema.findType(getOnlyElement(metadata.inputFields().iterator())), typeManager)) + .map(prestoType -> new KllHistogram(Slices.wrappedBuffer(blob), prestoType))); + } + + private static Types.NestedField getField(ColumnStatisticMetadata metadata, Table icebergTable, Snapshot snapshot) { - return Optional.ofNullable(icebergTable.schema().findField(metadata.getColumnName())).map(Types.NestedField::fieldId) + return Optional.ofNullable(icebergTable.schemas().get(snapshot.schemaId())) + .map(schema -> schema.findField(metadata.getColumnName())) .orElseThrow(() -> { log.warn("failed to find column name %s in schema of table %s", metadata.getColumnName(), icebergTable.name()); return new PrestoException(ICEBERG_INVALID_METADATA, format("failed to find column name %s in schema of table %s", metadata.getColumnName(), icebergTable.name())); @@ -577,7 +636,7 @@ private Map loadStatisticsFile(IcebergTableHandle tab if (value == null) { value = ColumnStatistics.builder(); } - statReader.read(metadata, blob, value); + statReader.read(metadata, blob, value, icebergTable, typeManager); return value; }); }); @@ -602,7 +661,7 @@ private Map loadStatisticsFile(IcebergTableHandle tab return finalResult.build(); } - public static List getSupportedColumnStatistics(String columnName, com.facebook.presto.common.type.Type type) + public static List getSupportedColumnStatistics(ConnectorSession session, String columnName, com.facebook.presto.common.type.Type type) { ImmutableList.Builder supportedStatistics = ImmutableList.builder(); // all types which support being passed to the sketch_theta function @@ -613,6 +672,16 @@ public static List getSupportedColumnStatistics(String columnName, format("RETURN sketch_theta(%s)", formatIdentifier(columnName)), ImmutableList.of(columnName))); } + if (isKllHistogramSupportedType(type)) { + String histogramFunctionFmt = "RETURN sketch_kll_with_k(%s, CAST(%s as bigint))"; + if (type instanceof DecimalType) { + histogramFunctionFmt = "RETURN sketch_kll_with_k(CAST(%s as double), CAST(%s as bigint))"; + } + supportedStatistics.add(HISTOGRAM.getColumnStatisticMetadataWithCustomFunction(columnName, + format(histogramFunctionFmt, formatIdentifier(columnName), getStatisticsKllSketchKParameter(session)), + ImmutableList.of(columnName))); + } + if (!(type instanceof FixedWidthType)) { supportedStatistics.add(TOTAL_SIZE_IN_BYTES.getColumnStatisticMetadata(columnName)); } diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/statistics/KllHistogram.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/statistics/KllHistogram.java new file mode 100644 index 000000000000..0aab4cc0d175 --- /dev/null +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/statistics/KllHistogram.java @@ -0,0 +1,210 @@ +/* + * 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.iceberg.statistics; + +import com.facebook.presto.common.type.AbstractIntType; +import com.facebook.presto.common.type.AbstractLongType; +import com.facebook.presto.common.type.AbstractVarcharType; +import com.facebook.presto.common.type.Type; +import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.statistics.ConnectorHistogram; +import com.facebook.presto.spi.statistics.Estimate; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.annotations.VisibleForTesting; +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; +import org.apache.datasketches.common.ArrayOfBooleansSerDe; +import org.apache.datasketches.common.ArrayOfDoublesSerDe; +import org.apache.datasketches.common.ArrayOfItemsSerDe; +import org.apache.datasketches.common.ArrayOfLongsSerDe; +import org.apache.datasketches.common.ArrayOfStringsSerDe; +import org.apache.datasketches.kll.KllItemsSketch; +import org.apache.datasketches.memory.Memory; +import org.openjdk.jol.info.ClassLayout; + +import java.util.Comparator; +import java.util.function.Function; + +import static com.facebook.presto.common.type.BooleanType.BOOLEAN; +import static com.facebook.presto.common.type.Decimals.isLongDecimal; +import static com.facebook.presto.common.type.Decimals.isShortDecimal; +import static com.facebook.presto.common.type.DoubleType.DOUBLE; +import static com.facebook.presto.common.type.RealType.REAL; +import static com.facebook.presto.common.type.SmallintType.SMALLINT; +import static com.facebook.presto.common.type.TinyintType.TINYINT; +import static com.facebook.presto.common.type.TypeUtils.isNumericType; +import static com.facebook.presto.spi.StandardErrorCode.INVALID_ARGUMENTS; +import static com.google.common.base.MoreObjects.toStringHelper; +import static com.google.common.base.Verify.verify; +import static java.nio.ByteOrder.LITTLE_ENDIAN; +import static java.util.Objects.requireNonNull; +import static org.apache.datasketches.quantilescommon.QuantileSearchCriteria.EXCLUSIVE; +import static org.apache.datasketches.quantilescommon.QuantileSearchCriteria.INCLUSIVE; + +public class KllHistogram + implements ConnectorHistogram +{ + private static final long INSTANCE_SIZE = ClassLayout.parseClass(KllHistogram.class).instanceSize(); + // since the actual type parameter is only known at runtime, we can't concretely specify it + private final KllItemsSketch sketch; + private final Type type; + private final Function toDouble; + private final Function fromDouble; + + @SuppressWarnings({"unchecked", "rawtypes"}) + @JsonCreator + public KllHistogram(@JsonProperty("sketch") Slice bytes, @JsonProperty("type") Type type) + { + verify(isKllHistogramSupportedType(type), "histograms do not currently support type " + type.getDisplayName()); + this.type = requireNonNull(type, "type is null"); + SketchParameters parameters = getSketchParameters(type); + // the actual sketch can only accept the same object types which generated it + // however, the API can only accept or generate double types. We cast the inputs + // and results to/from double to satisfy the underlying sketch type. + if (parameters.getSerde().getClassOfT().equals(Double.class)) { + toDouble = x -> (double) x; + fromDouble = x -> x; + } + else if (parameters.getSerde().getClassOfT().equals(Long.class)) { + // dual cast to auto-box/unbox from Double/Long for sketch + toDouble = x -> (double) (long) x; + fromDouble = x -> (long) (double) x; + } + else { + throw new PrestoException(INVALID_ARGUMENTS, "can't create kll sketch from type: " + type); + } + sketch = KllItemsSketch.wrap(Memory.wrap(bytes.toByteBuffer(), LITTLE_ENDIAN), parameters.getComparator(), parameters.getSerde()); + } + + public static boolean isKllHistogramSupportedType(Type type) + { + try { + return isNumericType(type) || + type instanceof AbstractIntType; + } + catch (PrestoException e) { + return false; + } + } + + @JsonProperty + public Slice getSketch() + { + return Slices.wrappedBuffer(sketch.toByteArray()); + } + + @JsonProperty + public Type getType() + { + return type; + } + + @VisibleForTesting + @SuppressWarnings("rawtypes") + public KllItemsSketch getKllSketch() + { + return sketch; + } + + @Override + public Estimate cumulativeProbability(double value, boolean inclusive) + { + return Estimate.of(sketch.getRank(fromDouble.apply(value), inclusive ? INCLUSIVE : EXCLUSIVE)); + } + + @Override + public Estimate inverseCumulativeProbability(double percentile) + { + return Estimate.of(toDouble.apply(sketch.getQuantile(percentile))); + } + + /** + * The memory utilization is dominated by the size of the sketch. This estimate + * doesn't account for the other fields in the class. + */ + @Override + public long getEstimatedSize() + { + return INSTANCE_SIZE + sketch.getSerializedSizeBytes(); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("type", type) + .add("k", this.sketch.getK()) + .add("N", this.sketch.getN()) + .add("retained", this.sketch.getNumRetained()) + .add("mingetSerialized", this.sketch.getMinItem()) + .add("max", this.sketch.getMaxItem()) + .add("p50", sketch.getQuantile(0.5)) + .add("p75", sketch.getQuantile(0.75)) + .add("p90", sketch.getQuantile(0.90)) + .add("p99", sketch.getQuantile(0.99)) + .add("p99.9", sketch.getQuantile(0.999)) + .toString(); + } + + private static class SketchParameters + { + private final Comparator comparator; + private final ArrayOfItemsSerDe serde; + + public SketchParameters(Comparator comparator, ArrayOfItemsSerDe serde) + { + this.comparator = comparator; + this.serde = serde; + } + + public Comparator getComparator() + { + return comparator; + } + + public ArrayOfItemsSerDe getSerde() + { + return serde; + } + } + + private static SketchParameters getSketchParameters(Type type) + { + if (type.equals(REAL)) { + return new SketchParameters<>(Double::compareTo, new ArrayOfDoublesSerDe()); + } + else if (isShortDecimal(type)) { + return new SketchParameters<>(Double::compareTo, new ArrayOfDoublesSerDe()); + } + else if (isLongDecimal(type)) { + return new SketchParameters<>(Double::compareTo, new ArrayOfDoublesSerDe()); + } + else if (type.equals(DOUBLE)) { + return new SketchParameters<>(Double::compareTo, new ArrayOfDoublesSerDe()); + } + else if (type.equals(BOOLEAN)) { + return new SketchParameters<>(Boolean::compareTo, new ArrayOfBooleansSerDe()); + } + else if (type instanceof AbstractIntType || type instanceof AbstractLongType || type.equals(SMALLINT) || type.equals(TINYINT)) { + return new SketchParameters<>(Long::compareTo, new ArrayOfLongsSerDe()); + } + else if (type instanceof AbstractVarcharType) { + return new SketchParameters<>(String::compareTo, new ArrayOfStringsSerDe()); + } + else { + throw new PrestoException(INVALID_ARGUMENTS, "Unsupported type for KLL sketch: " + type); + } + } +} diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/IcebergDistributedTestBase.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/IcebergDistributedTestBase.java index 95e35fbcece0..ebc9814c774b 100644 --- a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/IcebergDistributedTestBase.java +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/IcebergDistributedTestBase.java @@ -21,6 +21,9 @@ import com.facebook.presto.common.transaction.TransactionId; import com.facebook.presto.common.type.FixedWidthType; import com.facebook.presto.common.type.TimeZoneKey; +import com.facebook.presto.common.type.Type; +import com.facebook.presto.common.type.TypeParameter; +import com.facebook.presto.hive.BaseHiveColumnHandle; import com.facebook.presto.hive.HdfsConfiguration; import com.facebook.presto.hive.HdfsConfigurationInitializer; import com.facebook.presto.hive.HdfsContext; @@ -41,6 +44,8 @@ import com.facebook.presto.spi.connector.classloader.ClassLoaderSafeConnectorMetadata; import com.facebook.presto.spi.security.AllowAllAccessControl; import com.facebook.presto.spi.statistics.ColumnStatistics; +import com.facebook.presto.spi.statistics.ConnectorHistogram; +import com.facebook.presto.spi.statistics.DoubleRange; import com.facebook.presto.spi.statistics.Estimate; import com.facebook.presto.spi.statistics.TableStatistics; import com.facebook.presto.testing.MaterializedResult; @@ -54,6 +59,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Maps; +import org.apache.commons.math3.distribution.NormalDistribution; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -101,17 +107,23 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.UUID; import java.util.function.BiConsumer; +import java.util.function.Consumer; import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; import static com.facebook.presto.SystemSessionProperties.LEGACY_TIMESTAMP; +import static com.facebook.presto.SystemSessionProperties.OPTIMIZER_USE_HISTOGRAMS; import static com.facebook.presto.common.type.BigintType.BIGINT; +import static com.facebook.presto.common.type.DateType.DATE; +import static com.facebook.presto.common.type.DoubleType.DOUBLE; import static com.facebook.presto.common.type.IntegerType.INTEGER; +import static com.facebook.presto.common.type.RealType.REAL; import static com.facebook.presto.common.type.TimeZoneKey.UTC_KEY; import static com.facebook.presto.common.type.VarcharType.VARCHAR; import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.SYNTHESIZED; @@ -135,15 +147,20 @@ import static com.facebook.presto.testing.TestingConnectorSession.SESSION; import static com.facebook.presto.testing.assertions.Assert.assertEquals; import static com.facebook.presto.tests.sql.TestTable.randomTableSuffix; +import static com.facebook.presto.type.DecimalParametricType.DECIMAL; +import static com.google.common.collect.ImmutableMap.toImmutableMap; import static com.google.common.io.Files.createTempDir; import static java.lang.String.format; import static java.util.Objects.requireNonNull; import static java.util.UUID.randomUUID; +import static java.util.function.Function.identity; import static org.apache.iceberg.SnapshotSummary.TOTAL_DATA_FILES_PROP; import static org.apache.iceberg.SnapshotSummary.TOTAL_DELETE_FILES_PROP; import static org.apache.parquet.column.ParquetProperties.WriterVersion.PARQUET_1_0; import static org.apache.parquet.column.ParquetProperties.WriterVersion.PARQUET_2_0; +import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotEquals; +import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; @Test(singleThreaded = true) @@ -1026,6 +1043,11 @@ private TableStatistics getTableStats(String name, Optional snapshot) return getTableStats(name, snapshot, getSession(), Optional.empty()); } + private TableStatistics getTableStats(String name, Optional snapshot, Session session) + { + return getTableStats(name, snapshot, session, Optional.empty()); + } + private TableStatistics getTableStats(String name, Optional snapshot, Session session, Optional> columns) { TransactionId transactionId = getQueryRunner().getTransactionManager().beginTransaction(false); @@ -1453,6 +1475,59 @@ public void testMetadataDeleteOnUnPartitionedTableWithDeleteFiles() } } + @DataProvider(name = "validHistogramTypes") + public Object[][] validHistogramTypesDataProvider() + { + return new Object[][] { + // types not supported in Iceberg connector, but that histogram could support + // {TINYINT, new String[]{"1", "2", "10"}}, + // {SMALLINT, new String[]{"1", "2", "10"}}, + // {TIMESTAMP_WITH_TIME_ZONE, new String[]{"now() + interval '1' hour", "now() + interval '2' hour"}}, + // iceberg stores microsecond precision but presto calculates on millisecond precision + // need a fix to properly convert for the optimizer. + // {TIMESTAMP, new String[] {"localtimestamp + interval '1' hour", "localtimestamp + interval '2' hour"}}, + // {TIME, new String[] {"localtime", "localtime + interval '1' hour"}}, + // supported types + {INTEGER, new String[] {"1", "5", "9"}}, + {BIGINT, new String[] {"2", "4", "6"}}, + {DOUBLE, new String[] {"1.0", "3.1", "4.6"}}, + // short decimal + {DECIMAL.createType(ImmutableList.of(TypeParameter.of(2L), TypeParameter.of(1L))), new String[] {"0.0", "3.0", "4.0"}}, + // long decimal + {DECIMAL.createType(ImmutableList.of(TypeParameter.of(38L), TypeParameter.of(1L))), new String[] {"0.0", "3.0", "4.0"}}, + {DATE, new String[] {"date '2024-01-01'", "date '2024-03-30'", "date '2024-05-30'"}}, + {REAL, new String[] {"1.0", "2.0", "3.0"}}, + }; + } + + /** + * Verifies that the histogram is returned after ANALYZE for a variety of types + */ + @Test(dataProvider = "validHistogramTypes") + public void testHistogramStorage(Type type, Object[] values) + { + try { + Session session = Session.builder(getSession()) + .setSystemProperty(OPTIMIZER_USE_HISTOGRAMS, "true") + .build(); + assertQuerySucceeds("DROP TABLE IF EXISTS create_histograms"); + assertQuerySucceeds(String.format("CREATE TABLE create_histograms (c %s)", type.getDisplayName())); + assertQuerySucceeds(String.format("INSERT INTO create_histograms VALUES %s", Joiner.on(", ").join(values))); + assertQuerySucceeds(session, "ANALYZE create_histograms"); + TableStatistics tableStatistics = getTableStats("create_histograms"); + Map nameToHandle = tableStatistics.getColumnStatistics().keySet() + .stream().map(IcebergColumnHandle.class::cast) + .collect(Collectors.toMap(BaseHiveColumnHandle::getName, identity())); + assertNotNull(nameToHandle.get("c")); + IcebergColumnHandle handle = nameToHandle.get("c"); + ColumnStatistics statistics = tableStatistics.getColumnStatistics().get(handle); + assertTrue(statistics.getHistogram().isPresent()); + } + finally { + assertQuerySucceeds("DROP TABLE IF EXISTS create_histograms"); + } + } + @Test public void testMetadataDeleteOnPartitionedTableWithDeleteFiles() { @@ -1998,6 +2073,151 @@ public void testBatchReadOnTimeType(WriterVersion writerVersion) assertQuerySucceeds("DROP TABLE time_batch_read"); } + public void testAllNullHistogramColumn() + { + try { + Session session = Session.builder(getSession()) + .setSystemProperty(OPTIMIZER_USE_HISTOGRAMS, "true") + .build(); + assertQuerySucceeds("DROP TABLE IF EXISTS histogram_all_nulls"); + assertQuerySucceeds("CREATE TABLE histogram_all_nulls (c bigint)"); + TableStatistics stats = getTableStats("histogram_all_nulls", Optional.empty(), session); + assertFalse(stats.getColumnStatistics().values().stream().findFirst().isPresent()); + assertUpdate("INSERT INTO histogram_all_nulls VALUES NULL, NULL, NULL, NULL, NULL", 5); + stats = getTableStats("histogram_all_nulls", Optional.empty(), session); + assertFalse(stats.getColumnStatistics().values().stream().findFirst() + .get().getHistogram().isPresent()); + assertQuerySucceeds(session, "ANALYZE histogram_all_nulls"); + stats = getTableStats("histogram_all_nulls", Optional.empty(), session); + assertFalse(stats.getColumnStatistics().values().stream().findFirst() + .get().getHistogram().isPresent()); + } + finally { + assertQuerySucceeds("DROP TABLE IF EXISTS histogram_all_nulls"); + } + } + + @Test(dataProvider = "validHistogramTypes") + public void testHistogramShowStats(Type type, Object[] values) + { + try { + Session session = Session.builder(getSession()) + .setSystemProperty(OPTIMIZER_USE_HISTOGRAMS, "true") + .build(); + assertQuerySucceeds("DROP TABLE IF EXISTS create_histograms"); + assertQuerySucceeds(String.format("CREATE TABLE show_histograms (c %s)", type.getDisplayName())); + assertQuerySucceeds(String.format("INSERT INTO show_histograms VALUES %s", Joiner.on(", ").join(values))); + assertQuerySucceeds(session, "ANALYZE show_histograms"); + TableStatistics tableStatistics = getTableStats("show_histograms", Optional.empty(), session, Optional.empty()); + Map> histogramByColumnName = tableStatistics.getColumnStatistics() + .entrySet() + .stream() + .collect(toImmutableMap( + entry -> ((IcebergColumnHandle) entry.getKey()).getName(), + entry -> entry.getValue().getHistogram())); + MaterializedResult stats = getQueryRunner().execute("SHOW STATS for show_histograms"); + stats.getMaterializedRows() + .forEach(row -> { + String name = (String) row.getField(0); + String histogram = (String) row.getField(7); + assertEquals(Optional.ofNullable(histogramByColumnName.get(name)) + .flatMap(identity()) + .map(Objects::toString).orElse(null), + histogram); + }); + } + finally { + assertQuerySucceeds("DROP TABLE IF EXISTS show_histograms"); + } + } + + /** + * Verifies that when the users opts-in to using histograms that the + * optimizer estimates reflect the actual dataset for a variety of filter + * types (LTE, GT, EQ, NE) on a non-uniform data distribution + */ + @Test + public void testHistogramsUsedInOptimization() + { + Session histogramSession = Session.builder(getSession()) + .setSystemProperty(OPTIMIZER_USE_HISTOGRAMS, "true") + .build(); + // standard-normal distribution should have vastly different estimates than uniform at the tails (e.g. -3, +3) + NormalDistribution dist = new NormalDistribution(0, 1); + double[] values = dist.sample(1000); + Arrays.sort(values); + + try { + assertQuerySucceeds("DROP TABLE IF EXISTS histogram_validation"); + assertQuerySucceeds("CREATE TABLE histogram_validation (c double)"); + assertQuerySucceeds(String.format("INSERT INTO histogram_validation VALUES %s", Joiner.on(", ").join(Arrays.stream(values).iterator()))); + assertQuerySucceeds(histogramSession, "ANALYZE histogram_validation"); + Consumer assertFilters = (value) -> { + // use Math.abs because if the value isn't found, the returned value of binary + // search is (- insert index). The absolute value index tells us roughly how + // many records would have been returned regardless of if the actual value is in the + // dataset + double estimatedRowCount = Math.abs(Arrays.binarySearch(values, value)); + assertPlan(histogramSession, "SELECT * FROM histogram_validation WHERE c <= " + value, + output(anyTree(tableScan("histogram_validation"))).withApproximateOutputRowCount(estimatedRowCount, 25)); + // check that inverse filter equals roughly the inverse number of rows + assertPlan(histogramSession, "SELECT * FROM histogram_validation WHERE c > " + value, + output(anyTree(tableScan("histogram_validation"))).withApproximateOutputRowCount(Math.max(0.0, values.length - estimatedRowCount), 25)); + // having an exact random double value from the distribution exist more than once is exceedingly rare. + // the histogram calculation should return 1 (and the inverse) in both situations + assertPlan(histogramSession, "SELECT * FROM histogram_validation WHERE c = " + value, + output(anyTree(tableScan("histogram_validation"))).withApproximateOutputRowCount(1.0, 25)); + assertPlan(histogramSession, "SELECT * FROM histogram_validation WHERE c != " + value, + output(anyTree(tableScan("histogram_validation"))).withApproximateOutputRowCount(values.length - 1, 25)); + }; + + assertFilters.accept(values[1]); // choose 1 greater than the min value + assertFilters.accept(-2.0); // should be very unlikely to generate a distribution where all values > -2.0 + assertFilters.accept(-1.0); + assertFilters.accept(0.0); + assertFilters.accept(1.0); + assertFilters.accept(2.0); // should be very unlikely to generate a distribution where all values < 2.0 + assertFilters.accept(values[values.length - 2]); // choose 1 less than the max value + } + finally { + assertQuerySucceeds("DROP TABLE IF EXISTS histogram_validation"); + } + } + + /** + * Verifies that the data in the histogram matches the mins/maxs of the values + * in the table when created + */ + @Test(dataProvider = "validHistogramTypes") + public void testHistogramReconstruction(Type type, Object[] values) + { + try { + Session session = Session.builder(getSession()) + .setSystemProperty(OPTIMIZER_USE_HISTOGRAMS, "true") + .build(); + assertQuerySucceeds("DROP TABLE IF EXISTS verify_histograms"); + assertQuerySucceeds(String.format("CREATE TABLE verify_histograms (c %s)", type.getDisplayName())); + assertQuerySucceeds(String.format("INSERT INTO verify_histograms VALUES %s", Joiner.on(", ").join(values))); + assertQuerySucceeds(session, "ANALYZE verify_histograms"); + TableStatistics tableStatistics = getTableStats("verify_histograms", Optional.empty(), session, Optional.empty()); + Map nameToHandle = tableStatistics.getColumnStatistics().keySet() + .stream().map(IcebergColumnHandle.class::cast) + .collect(Collectors.toMap(BaseHiveColumnHandle::getName, identity())); + assertNotNull(nameToHandle.get("c")); + IcebergColumnHandle handle = nameToHandle.get("c"); + ColumnStatistics statistics = tableStatistics.getColumnStatistics().get(handle); + ConnectorHistogram histogram = statistics.getHistogram().get(); + DoubleRange range = statistics.getRange().get(); + double min = range.getMin(); + double max = range.getMax(); + assertEquals(histogram.inverseCumulativeProbability(0.0).getValue(), min); + assertEquals(histogram.inverseCumulativeProbability(1.0).getValue(), max); + } + finally { + assertQuerySucceeds("DROP TABLE IF EXISTS verify_histograms"); + } + } + private void testCheckDeleteFiles(Table icebergTable, int expectedSize, List expectedFileContent) { // check delete file list diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergConfig.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergConfig.java index d28503a27d31..588b7273d44c 100644 --- a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergConfig.java +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergConfig.java @@ -69,7 +69,8 @@ public void testDefaults() .setMetadataPreviousVersionsMax(METADATA_PREVIOUS_VERSIONS_MAX_DEFAULT) .setMetadataDeleteAfterCommit(METADATA_DELETE_AFTER_COMMIT_ENABLED_DEFAULT) .setMetricsMaxInferredColumn(METRICS_MAX_INFERRED_COLUMN_DEFAULTS_DEFAULT) - .setMaxStatisticsFileCacheSize(succinctDataSize(256, MEGABYTE))); + .setMaxStatisticsFileCacheSize(succinctDataSize(256, MEGABYTE)) + .setStatisticsKllSketchKParameter(1024)); } @Test @@ -101,6 +102,7 @@ public void testExplicitPropertyMappings() .put("iceberg.metadata-delete-after-commit", "true") .put("iceberg.metrics-max-inferred-column", "16") .put("iceberg.max-statistics-file-cache-size", "512MB") + .put("iceberg.statistics-kll-sketch-k-parameter", "4096") .build(); IcebergConfig expected = new IcebergConfig() @@ -128,7 +130,8 @@ public void testExplicitPropertyMappings() .setMetadataPreviousVersionsMax(1) .setMetadataDeleteAfterCommit(true) .setMetricsMaxInferredColumn(16) - .setMaxStatisticsFileCacheSize(succinctDataSize(512, MEGABYTE)); + .setMaxStatisticsFileCacheSize(succinctDataSize(512, MEGABYTE)) + .setStatisticsKllSketchKParameter(4096); assertFullMapping(properties, expected); } diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/statistics/TestKllHistogram.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/statistics/TestKllHistogram.java new file mode 100644 index 000000000000..2db83a1b8f0a --- /dev/null +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/statistics/TestKllHistogram.java @@ -0,0 +1,166 @@ +/* + * 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.iceberg.statistics; + +import com.facebook.presto.common.type.CharType; +import com.facebook.presto.common.type.Type; +import com.facebook.presto.common.type.VarcharType; +import com.google.common.base.VerifyException; +import io.airlift.slice.Slices; +import org.apache.datasketches.common.ArrayOfDoublesSerDe; +import org.apache.datasketches.common.ArrayOfLongsSerDe; +import org.apache.datasketches.kll.KllItemsSketch; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +import java.nio.ByteBuffer; +import java.util.stream.DoubleStream; +import java.util.stream.LongStream; + +import static com.facebook.presto.common.type.BigintType.BIGINT; +import static com.facebook.presto.common.type.BooleanType.BOOLEAN; +import static com.facebook.presto.common.type.DateType.DATE; +import static com.facebook.presto.common.type.DecimalType.createDecimalType; +import static com.facebook.presto.common.type.DoubleType.DOUBLE; +import static com.facebook.presto.common.type.IntegerType.INTEGER; +import static com.facebook.presto.common.type.RealType.REAL; +import static com.facebook.presto.common.type.TimeType.TIME; +import static com.facebook.presto.common.type.TimestampType.TIMESTAMP; +import static com.facebook.presto.common.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; +import static com.facebook.presto.common.type.VarcharType.VARCHAR; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertThrows; + +public class TestKllHistogram +{ + @SuppressWarnings("unchecked") + @Test + public void testSimpleCreation() + { + KllItemsSketch sketch = KllItemsSketch.newHeapInstance(Double::compareTo, new ArrayOfDoublesSerDe()); + DoubleStream.iterate(0.0, i -> i + 1).limit(100).forEach(sketch::update); + KllHistogram histogram = new KllHistogram(Slices.wrappedBuffer(ByteBuffer.wrap(sketch.toByteArray())), DOUBLE); + assertSketchesEqual(histogram.getKllSketch(), sketch); + } + + @Test + public void smokeTestHistogram() + { + // a histogram with a uniform distribution from 0.0 to 99.0 + KllHistogram basicHistogram = generateDoublesHistogram(); + // inverse cumulative probability + assertEquals(basicHistogram.inverseCumulativeProbability(0.0).getValue(), 0.0, 1E-8); + assertEquals(basicHistogram.inverseCumulativeProbability(1.0).getValue(), 99.0, 1E-8); + assertEquals(basicHistogram.inverseCumulativeProbability(0.5).getValue(), 49.0, 1E-8); + + // cumulative probability w/ inclusivities + assertEquals(basicHistogram.cumulativeProbability(0.0, true).getValue(), 0.01, 1E-8); + assertEquals(basicHistogram.cumulativeProbability(0.0, false).getValue(), 0.0, 1E-8); + + assertEquals(basicHistogram.cumulativeProbability(49.0, false).getValue(), 0.49, 1E-8); + assertEquals(basicHistogram.cumulativeProbability(49.0, true).getValue(), 0.5, 1E-8); + + assertEquals(basicHistogram.cumulativeProbability(99.0, false).getValue(), 0.99, 1E-8); + assertEquals(basicHistogram.cumulativeProbability(99.0, true).getValue(), 1.0, 1E-8); + } + + @DataProvider(name = "kllSupportedTypes") + public static Object[][] kllHistogramTypeDataProvider() + { + return new Object[][] { + // long decimal (represented by Slice.class), currently not supported + // {createDecimalType(), TestKllHistogram.generateLongSketch()}, + // time and timestamp types need additional changes because iceberg stores them in + // microsecond format but Presto always processes in milliseconds + // {TIMESTAMP_WITH_TIME_ZONE, generateLongSketch()}, + // {TIMESTAMP_MICROSECONDS, generateLongSketch()}, + // {TIMESTAMP, generateLongSketch()}, + // {TIME, generateLongSketch()}, + {INTEGER, TestKllHistogram.generateLongSketch()}, + {BIGINT, TestKllHistogram.generateLongSketch()}, + {DOUBLE, TestKllHistogram.generateDoubleSketch()}, + {createDecimalType(3, 1), TestKllHistogram.generateDoubleSketch()}, + {DATE, TestKllHistogram.generateLongSketch()}, + {createDecimalType(38, 0), TestKllHistogram.generateDoubleSketch()}, + {REAL, generateDoubleSketch()}, + }; + } + + @DataProvider(name = "kllUnsupportedTypes") + public static Object[][] unsupportedKllHistogramTypes() + { + return new Object[][] { + // long decimal (represented by Slice.class), currently not supported + {CharType.createCharType(0)}, + {CharType.createCharType(100)}, + {BOOLEAN}, + {VARCHAR}, + {VarcharType.createVarcharType(10)}, + {TIMESTAMP}, + {TIMESTAMP_WITH_TIME_ZONE}, + {TIME}, + }; + } + + @SuppressWarnings("rawtypes") + @Test(dataProvider = "kllSupportedTypes") + public void testTypeCreation(Type type, KllItemsSketch sketch) + { + KllHistogram histogram = new KllHistogram(Slices.wrappedBuffer(sketch.toByteArray()), type); + double value = histogram.inverseCumulativeProbability(0.5).getValue(); + double probability = histogram.cumulativeProbability(49.0, true).getValue(); + assertEquals(probability, 0.5); + assertEquals(value, 49.0); + } + + @Test(dataProvider = "kllUnsupportedTypes") + public void testUnsupportedKllTypes(Type type) + { + assertThrows(VerifyException.class, () -> { + new KllHistogram(null, type); + }); + } + + /** + * @return generates a histogram of doubles from [0.0, 99.9] in intervals of 1.0 + */ + private static KllHistogram generateDoublesHistogram() + { + return new KllHistogram(Slices.wrappedBuffer(ByteBuffer.wrap(generateDoubleSketch().toByteArray())), DOUBLE); + } + + private static KllItemsSketch generateLongSketch() + { + KllItemsSketch sketch = KllItemsSketch.newHeapInstance(Long::compareTo, new ArrayOfLongsSerDe()); + LongStream.iterate(0, i -> i + 1).limit(100).forEach(sketch::update); + return sketch; + } + + private static KllItemsSketch generateDoubleSketch() + { + KllItemsSketch sketch = KllItemsSketch.newHeapInstance(Double::compareTo, new ArrayOfDoublesSerDe()); + DoubleStream.iterate(0.0, i -> i + 1).limit(100).forEach(sketch::update); + return sketch; + } + + private static void assertSketchesEqual(KllItemsSketch sketch, KllItemsSketch other) + { + assertEquals(other.getK(), sketch.getK()); + assertEquals(other.getN(), sketch.getN()); + assertEquals(other.getMinItem(), sketch.getMinItem()); + assertEquals(other.getMaxItem(), sketch.getMaxItem()); + assertEquals(other.getSortedView().getCumulativeWeights(), sketch.getSortedView().getCumulativeWeights()); + assertEquals(other.getSortedView().getQuantiles(), sketch.getSortedView().getQuantiles()); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/cost/ComparisonStatsCalculator.java b/presto-main/src/main/java/com/facebook/presto/cost/ComparisonStatsCalculator.java index 306e00cfb907..efc05bf4a20a 100644 --- a/presto-main/src/main/java/com/facebook/presto/cost/ComparisonStatsCalculator.java +++ b/presto-main/src/main/java/com/facebook/presto/cost/ComparisonStatsCalculator.java @@ -17,7 +17,10 @@ import com.facebook.presto.Session; import com.facebook.presto.SystemSessionProperties; import com.facebook.presto.spi.relation.VariableReferenceExpression; +import com.facebook.presto.spi.statistics.DisjointRangeDomainHistogram; import com.facebook.presto.spi.statistics.Estimate; +import com.facebook.presto.spi.statistics.HistogramCalculator; +import com.facebook.presto.spi.statistics.UniformDistributionHistogram; import com.facebook.presto.sql.tree.ComparisonExpression; import java.util.Optional; @@ -156,7 +159,7 @@ private PlanNodeStatsEstimate estimateFilterRange( .setStatisticsRange(intersectRange) .setNullsFraction(0.0); if (useHistograms) { - symbolNewEstimate.setHistogram(expressionStatistics.getHistogram().map(expressionHistogram -> DisjointRangeDomainHistogram.addConjunction(expressionHistogram, intersectRange))); + symbolNewEstimate.setHistogram(expressionStatistics.getHistogram().map(expressionHistogram -> DisjointRangeDomainHistogram.addConjunction(expressionHistogram, intersectRange.toPrestoRange()))); } estimate = estimate.mapVariableColumnStatistics(expressionVariable.get(), oldStats -> symbolNewEstimate.build()); @@ -171,7 +174,7 @@ private double calculateFilterFactor(VariableStatsEstimate variableStatistics, S Estimate filterEstimate; if (useHistograms) { Estimate distinctEstimate = isNaN(variableStatistics.getDistinctValuesCount()) ? Estimate.unknown() : Estimate.of(variableRange.getDistinctValuesCount()); - filterEstimate = HistogramCalculator.calculateFilterFactor(intersectRange, variableStatistics.getHistogram().orElse(new UniformDistributionHistogram(variableStatistics.getLowValue(), variableStatistics.getHighValue())), distinctEstimate, true); + filterEstimate = HistogramCalculator.calculateFilterFactor(intersectRange.toPrestoRange(), intersectRange.getDistinctValuesCount(), variableStatistics.getHistogram().orElse(new UniformDistributionHistogram(variableStatistics.getLowValue(), variableStatistics.getHighValue())), distinctEstimate, true); if (log.isDebugEnabled()) { double expressionFilter = variableRange.overlapPercentWith(intersectRange); if (!Double.isNaN(expressionFilter) && diff --git a/presto-main/src/main/java/com/facebook/presto/cost/ConnectorFilterStatsCalculatorService.java b/presto-main/src/main/java/com/facebook/presto/cost/ConnectorFilterStatsCalculatorService.java index 7a2e804aa7e1..85a51c2b6bb5 100644 --- a/presto-main/src/main/java/com/facebook/presto/cost/ConnectorFilterStatsCalculatorService.java +++ b/presto-main/src/main/java/com/facebook/presto/cost/ConnectorFilterStatsCalculatorService.java @@ -141,6 +141,11 @@ private static ColumnStatistics toColumnStatistics(VariableStatsEstimate variabl if (!Double.isNaN(variableStatsEstimate.getLowValue()) && !Double.isNaN(variableStatsEstimate.getHighValue())) { builder.setRange(new DoubleRange(variableStatsEstimate.getLowValue(), variableStatsEstimate.getHighValue())); } + + if (variableStatsEstimate.getHistogram().isPresent()) { + builder.setHistogram(variableStatsEstimate.getHistogram()); + } + return builder.build(); } } diff --git a/presto-main/src/main/java/com/facebook/presto/cost/JoinStatsRule.java b/presto-main/src/main/java/com/facebook/presto/cost/JoinStatsRule.java index 77e0b487a79d..b9f340b623ef 100644 --- a/presto-main/src/main/java/com/facebook/presto/cost/JoinStatsRule.java +++ b/presto-main/src/main/java/com/facebook/presto/cost/JoinStatsRule.java @@ -34,10 +34,10 @@ import static com.facebook.presto.SystemSessionProperties.getDefaultJoinSelectivityCoefficient; import static com.facebook.presto.SystemSessionProperties.shouldOptimizerUseHistograms; -import static com.facebook.presto.cost.DisjointRangeDomainHistogram.addConjunction; import static com.facebook.presto.cost.FilterStatsCalculator.UNKNOWN_FILTER_COEFFICIENT; import static com.facebook.presto.cost.VariableStatsEstimate.buildFrom; import static com.facebook.presto.expressions.LogicalRowExpressions.extractConjuncts; +import static com.facebook.presto.spi.statistics.DisjointRangeDomainHistogram.addConjunction; import static com.facebook.presto.sql.analyzer.ExpressionTreeUtils.getNodeLocation; import static com.facebook.presto.sql.planner.plan.Patterns.join; import static com.facebook.presto.sql.tree.ComparisonExpression.Operator.EQUAL; @@ -250,7 +250,7 @@ private PlanNodeStatsEstimate filterByAuxiliaryClause(PlanNodeStatsEstimate stat .setStatisticsRange(intersect) .setDistinctValuesCount(retainedNdv); if (useHistograms) { - newLeftStats.setHistogram(leftStats.getHistogram().map(leftHistogram -> addConjunction(leftHistogram, intersect))); + newLeftStats.setHistogram(leftStats.getHistogram().map(leftHistogram -> addConjunction(leftHistogram, intersect.toPrestoRange()))); } VariableStatsEstimate.Builder newRightStats = buildFrom(rightStats) @@ -258,7 +258,7 @@ private PlanNodeStatsEstimate filterByAuxiliaryClause(PlanNodeStatsEstimate stat .setStatisticsRange(intersect) .setDistinctValuesCount(retainedNdv); if (useHistograms) { - newRightStats.setHistogram(rightStats.getHistogram().map(rightHistogram -> addConjunction(rightHistogram, intersect))); + newRightStats.setHistogram(rightStats.getHistogram().map(rightHistogram -> addConjunction(rightHistogram, intersect.toPrestoRange()))); } PlanNodeStatsEstimate.Builder result = PlanNodeStatsEstimate.buildFrom(stats) diff --git a/presto-main/src/main/java/com/facebook/presto/cost/PlanNodeStatsEstimateMath.java b/presto-main/src/main/java/com/facebook/presto/cost/PlanNodeStatsEstimateMath.java index 1b2797e18a8a..2a280ae2524b 100644 --- a/presto-main/src/main/java/com/facebook/presto/cost/PlanNodeStatsEstimateMath.java +++ b/presto-main/src/main/java/com/facebook/presto/cost/PlanNodeStatsEstimateMath.java @@ -14,10 +14,11 @@ package com.facebook.presto.cost; import com.facebook.presto.spi.statistics.ConnectorHistogram; +import com.facebook.presto.spi.statistics.DisjointRangeDomainHistogram; import java.util.Optional; -import static com.facebook.presto.cost.DisjointRangeDomainHistogram.addConjunction; +import static com.facebook.presto.spi.statistics.DisjointRangeDomainHistogram.addConjunction; import static com.google.common.base.Preconditions.checkArgument; import static java.lang.Double.NaN; import static java.lang.Double.isNaN; @@ -139,7 +140,7 @@ public PlanNodeStatsEstimate capStats(PlanNodeStatsEstimate stats, PlanNodeStats double cappedNullsFraction = cappedRowCount == 0 ? 1 : cappedNumberOfNulls / cappedRowCount; newSymbolStats.setNullsFraction(cappedNullsFraction); if (shouldUseHistograms) { - newSymbolStats.setHistogram(symbolStats.getHistogram().map(symbolHistogram -> addConjunction(symbolHistogram, new StatisticRange(newLow, newHigh, 0)))); + newSymbolStats.setHistogram(symbolStats.getHistogram().map(symbolHistogram -> addConjunction(symbolHistogram, new StatisticRange(newLow, newHigh, 0).toPrestoRange()))); } result.addVariableStatistics(symbol, newSymbolStats.build()); @@ -296,8 +297,8 @@ private VariableStatsEstimate addColumnStats( .setNullsFraction(newNullsFraction); if (shouldUseHistograms) { Optional newHistogram = RangeAdditionStrategy.INTERSECT == strategy ? - leftStats.getHistogram().map(leftHistogram -> DisjointRangeDomainHistogram.addConjunction(leftHistogram, rightRange)) : - leftStats.getHistogram().map(leftHistogram -> DisjointRangeDomainHistogram.addDisjunction(leftHistogram, rightRange)); + leftStats.getHistogram().map(leftHistogram -> DisjointRangeDomainHistogram.addConjunction(leftHistogram, rightRange.toPrestoRange())) : + leftStats.getHistogram().map(leftHistogram -> DisjointRangeDomainHistogram.addDisjunction(leftHistogram, rightRange.toPrestoRange())); statistics.setHistogram(newHistogram); } diff --git a/presto-main/src/main/java/com/facebook/presto/cost/StatisticRange.java b/presto-main/src/main/java/com/facebook/presto/cost/StatisticRange.java index 4d80e13cb92c..060e02bc8b07 100644 --- a/presto-main/src/main/java/com/facebook/presto/cost/StatisticRange.java +++ b/presto-main/src/main/java/com/facebook/presto/cost/StatisticRange.java @@ -13,19 +13,19 @@ */ package com.facebook.presto.cost; +import com.facebook.presto.common.predicate.Range; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.collect.BoundType; -import com.google.common.collect.Range; import java.util.Objects; +import static com.facebook.presto.common.type.DoubleType.DOUBLE; +import static com.facebook.presto.spi.statistics.ColumnStatistics.INFINITE_TO_FINITE_RANGE_INTERSECT_OVERLAP_HEURISTIC_FACTOR; +import static com.facebook.presto.spi.statistics.ColumnStatistics.INFINITE_TO_INFINITE_RANGE_INTERSECT_OVERLAP_HEURISTIC_FACTOR; import static com.facebook.presto.util.MoreMath.nearlyEqual; import static com.google.common.base.MoreObjects.toStringHelper; import static com.google.common.base.Preconditions.checkArgument; -import static java.lang.Double.NEGATIVE_INFINITY; import static java.lang.Double.NaN; -import static java.lang.Double.POSITIVE_INFINITY; import static java.lang.Double.isFinite; import static java.lang.Double.isInfinite; import static java.lang.Double.isNaN; @@ -36,9 +36,6 @@ public class StatisticRange { - protected static final double INFINITE_TO_FINITE_RANGE_INTERSECT_OVERLAP_HEURISTIC_FACTOR = 0.25; - protected static final double INFINITE_TO_INFINITE_RANGE_INTERSECT_OVERLAP_HEURISTIC_FACTOR = 0.5; - // TODO unify field and method names with SymbolStatsEstimate /** * {@code NaN} represents empty range ({@code high} must be {@code NaN} too) @@ -222,19 +219,12 @@ public StatisticRange addAndCollapseDistinctValues(StatisticRange other) return expandRangeWithNewDistinct(newDistinctValues, other); } - public Range toRange() + public Range toPrestoRange() { - return Range.range(low, openLow ? BoundType.OPEN : BoundType.CLOSED, high, openHigh ? BoundType.OPEN : BoundType.CLOSED); - } - - public static StatisticRange fromRange(Range range) - { - return new StatisticRange( - range.hasLowerBound() ? range.lowerEndpoint() : NEGATIVE_INFINITY, - !range.hasLowerBound() || range.lowerBoundType() == BoundType.OPEN, - range.hasUpperBound() ? range.upperEndpoint() : POSITIVE_INFINITY, - !range.hasUpperBound() || range.upperBoundType() == BoundType.OPEN, - NaN); + if (low == high) { + return Range.equal(DOUBLE, low); + } + return Range.range(DOUBLE, low, !openLow, high, !openHigh); } private StatisticRange expandRangeWithNewDistinct(double newDistinctValues, StatisticRange other) diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/BasePlanFragmenter.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/BasePlanFragmenter.java index ceb0186ffb41..190271f3e39a 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/BasePlanFragmenter.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/BasePlanFragmenter.java @@ -62,7 +62,6 @@ import static com.facebook.presto.SystemSessionProperties.isForceSingleNodeOutput; import static com.facebook.presto.SystemSessionProperties.isSingleNodeExecutionEnabled; -import static com.facebook.presto.SystemSessionProperties.shouldOptimizerUseHistograms; import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED; import static com.facebook.presto.sql.TemporaryTableUtil.assignPartitioningVariables; import static com.facebook.presto.sql.TemporaryTableUtil.assignTemporaryTableColumnNames; @@ -122,7 +121,7 @@ public BasePlanFragmenter( this.idAllocator = requireNonNull(idAllocator, "idAllocator is null"); this.variableAllocator = requireNonNull(variableAllocator, "variableAllocator is null"); this.outputTableWriterNodeIds = ImmutableSet.copyOf(requireNonNull(outputTableWriterNodeIds, "outputTableWriterNodeIds is null")); - this.statisticsAggregationPlanner = new StatisticsAggregationPlanner(variableAllocator, metadata.getFunctionAndTypeManager(), session, shouldOptimizerUseHistograms(session)); + this.statisticsAggregationPlanner = new StatisticsAggregationPlanner(variableAllocator, metadata.getFunctionAndTypeManager(), session); } public SubPlan buildRootFragment(PlanNode root, FragmentProperties properties) 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 2b68e642dec9..9d0d1aaaf180 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 @@ -86,7 +86,6 @@ import java.util.Optional; import java.util.Set; -import static com.facebook.presto.SystemSessionProperties.shouldOptimizerUseHistograms; import static com.facebook.presto.common.type.BigintType.BIGINT; import static com.facebook.presto.common.type.VarbinaryType.VARBINARY; import static com.facebook.presto.metadata.MetadataUtil.getConnectorIdOrThrow; @@ -135,7 +134,7 @@ public LogicalPlanner( this.idAllocator = requireNonNull(idAllocator, "idAllocator is null"); this.metadata = requireNonNull(metadata, "metadata is null"); this.variableAllocator = requireNonNull(variableAllocator, "variableAllocator is null"); - this.statisticsAggregationPlanner = new StatisticsAggregationPlanner(this.variableAllocator, metadata.getFunctionAndTypeManager(), session, shouldOptimizerUseHistograms(session)); + this.statisticsAggregationPlanner = new StatisticsAggregationPlanner(this.variableAllocator, metadata.getFunctionAndTypeManager(), session); this.sqlParser = requireNonNull(sqlParser, "sqlParser is null"); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/StatisticsAggregationPlanner.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/StatisticsAggregationPlanner.java index 01da076c6807..dab0770f4788 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/StatisticsAggregationPlanner.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/StatisticsAggregationPlanner.java @@ -43,6 +43,7 @@ import java.util.Optional; import java.util.stream.Collectors; +import static com.facebook.presto.SystemSessionProperties.shouldOptimizerUseHistograms; import static com.facebook.presto.common.type.BigintType.BIGINT; import static com.facebook.presto.common.type.UnknownType.UNKNOWN; import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED; @@ -62,13 +63,13 @@ public class StatisticsAggregationPlanner private final Session session; private final FunctionAndTypeManager functionAndTypeManager; - public StatisticsAggregationPlanner(VariableAllocator variableAllocator, FunctionAndTypeManager functionAndTypeManager, Session session, boolean useHistograms) + public StatisticsAggregationPlanner(VariableAllocator variableAllocator, FunctionAndTypeManager functionAndTypeManager, Session session) { this.variableAllocator = requireNonNull(variableAllocator, "variableAllocator is null"); this.session = requireNonNull(session, "session is null"); this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionAndTypeManager is null"); this.functionAndTypeResolver = functionAndTypeManager.getFunctionAndTypeResolver(); - this.useHistograms = useHistograms; + this.useHistograms = shouldOptimizerUseHistograms(session); } public TableStatisticAggregation createStatisticsAggregation(TableStatisticsMetadata statisticsMetadata, Map columnToVariableMap) diff --git a/presto-main/src/main/java/com/facebook/presto/sql/rewrite/ShowStatsRewrite.java b/presto-main/src/main/java/com/facebook/presto/sql/rewrite/ShowStatsRewrite.java index 7a7396824e09..801cb6f33773 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/rewrite/ShowStatsRewrite.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/rewrite/ShowStatsRewrite.java @@ -21,6 +21,7 @@ import com.facebook.presto.common.type.IntegerType; import com.facebook.presto.common.type.RealType; import com.facebook.presto.common.type.SmallintType; +import com.facebook.presto.common.type.SqlTime; import com.facebook.presto.common.type.SqlTimestamp; import com.facebook.presto.common.type.TinyintType; import com.facebook.presto.common.type.Type; @@ -80,6 +81,7 @@ import static com.facebook.presto.common.type.SqlTimestamp.MICROSECONDS_PER_MILLISECOND; import static com.facebook.presto.common.type.StandardTypes.DOUBLE; import static com.facebook.presto.common.type.StandardTypes.VARCHAR; +import static com.facebook.presto.common.type.TimeType.TIME; import static com.facebook.presto.common.type.TimestampType.TIMESTAMP; import static com.facebook.presto.metadata.MetadataUtil.createQualifiedObjectName; import static com.facebook.presto.sql.QueryUtil.aliased; @@ -373,6 +375,9 @@ private Expression toStringLiteral(Type type, double value) if (type.equals(TIMESTAMP)) { return new StringLiteral(new SqlTimestamp(round(value) / MICROSECONDS_PER_MILLISECOND, session.getSqlFunctionProperties().getTimeZoneKey(), MILLISECONDS).toString()); } + if (type.equals(TIME)) { + return new StringLiteral(new SqlTime(round(value)).toString()); + } throw new IllegalArgumentException("Unexpected type: " + type); } } diff --git a/presto-main/src/test/java/com/facebook/presto/cost/TestHistogramCalculator.java b/presto-main/src/test/java/com/facebook/presto/cost/TestHistogramCalculator.java deleted file mode 100644 index ddccfdfe3c06..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/cost/TestHistogramCalculator.java +++ /dev/null @@ -1,100 +0,0 @@ -/* - * 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.cost; - -import com.facebook.presto.spi.statistics.ConnectorHistogram; -import com.facebook.presto.spi.statistics.Estimate; -import org.testng.annotations.Test; - -import static com.facebook.presto.cost.HistogramCalculator.calculateFilterFactor; -import static java.lang.Double.NEGATIVE_INFINITY; -import static java.lang.Double.NaN; -import static java.lang.Double.POSITIVE_INFINITY; -import static org.testng.Assert.assertEquals; - -public class TestHistogramCalculator -{ - @Test - public void testCalculateFilterFactor() - { - StatisticRange zeroToTen = range(0, 10, 10); - StatisticRange empty = StatisticRange.empty(); - - // Equal ranges - assertFilterFactor(Estimate.of(1.0), zeroToTen, uniformHist(0, 10), 5); - assertFilterFactor(Estimate.of(1.0), zeroToTen, uniformHist(0, 10), 20); - - // Some overlap - assertFilterFactor(Estimate.of(0.5), range(5, 3000, 5), uniformHist(zeroToTen), zeroToTen.getDistinctValuesCount()); - - // Single value overlap - assertFilterFactor(Estimate.of(1.0 / zeroToTen.getDistinctValuesCount()), range(3, 3, 1), uniformHist(zeroToTen), zeroToTen.getDistinctValuesCount()); - assertFilterFactor(Estimate.of(1.0 / zeroToTen.getDistinctValuesCount()), range(10, 100, 357), uniformHist(zeroToTen), zeroToTen.getDistinctValuesCount()); - - // No overlap - assertFilterFactor(Estimate.zero(), range(20, 30, 10), uniformHist(zeroToTen), zeroToTen.getDistinctValuesCount()); - - // Empty ranges - assertFilterFactor(Estimate.zero(), zeroToTen, uniformHist(empty), empty.getDistinctValuesCount()); - assertFilterFactor(Estimate.zero(), empty, uniformHist(zeroToTen), zeroToTen.getDistinctValuesCount()); - - // no test for (empty, empty) since any return value is correct - assertFilterFactor(Estimate.zero(), unboundedRange(10), uniformHist(empty), empty.getDistinctValuesCount()); - assertFilterFactor(Estimate.zero(), empty, uniformHist(unboundedRange(10)), 10); - - // Unbounded (infinite), NDV-based - assertFilterFactor(Estimate.of(0.5), unboundedRange(10), uniformHist(unboundedRange(20)), 20); - assertFilterFactor(Estimate.of(1.0), unboundedRange(20), uniformHist(unboundedRange(10)), 10); - - // NEW TESTS (TPC-H Q2) - // unbounded ranges - assertFilterFactor(Estimate.of(.5), unboundedRange(0.5), uniformHist(unboundedRange(NaN)), NaN); - // unbounded ranges with limited distinct values - assertFilterFactor(Estimate.of(0.2), unboundedRange(1.0), - domainConstrained(unboundedRange(5.0), uniformHist(unboundedRange(7.0))), 5.0); - } - - private static StatisticRange range(double low, double high, double distinctValues) - { - return new StatisticRange(low, high, distinctValues); - } - - private static StatisticRange unboundedRange(double distinctValues) - { - return new StatisticRange(NEGATIVE_INFINITY, POSITIVE_INFINITY, distinctValues); - } - - private static void assertFilterFactor(Estimate expected, StatisticRange range, ConnectorHistogram histogram, double totalDistinctValues) - { - assertEquals( - calculateFilterFactor(range, histogram, Estimate.estimateFromDouble(totalDistinctValues), true), - expected); - } - - private static ConnectorHistogram uniformHist(StatisticRange range) - { - return uniformHist(range.getLow(), range.getHigh()); - } - - private static ConnectorHistogram uniformHist(double low, double high) - { - return new UniformDistributionHistogram(low, high); - } - - private static ConnectorHistogram domainConstrained(StatisticRange range, ConnectorHistogram source) - { - return DisjointRangeDomainHistogram.addDisjunction(source, range); - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/cost/TestPlanNodeStatsEstimateMath.java b/presto-main/src/test/java/com/facebook/presto/cost/TestPlanNodeStatsEstimateMath.java index dc2e60bb46e8..79e47477bc12 100644 --- a/presto-main/src/test/java/com/facebook/presto/cost/TestPlanNodeStatsEstimateMath.java +++ b/presto-main/src/test/java/com/facebook/presto/cost/TestPlanNodeStatsEstimateMath.java @@ -15,6 +15,8 @@ import com.facebook.presto.spi.relation.VariableReferenceExpression; import com.facebook.presto.spi.statistics.ConnectorHistogram; +import com.facebook.presto.spi.statistics.DisjointRangeDomainHistogram; +import com.facebook.presto.spi.statistics.UniformDistributionHistogram; import org.testng.annotations.Test; import java.util.Optional; @@ -374,31 +376,31 @@ public void testAddHistograms() assertEquals(calculator.addStatsAndCollapseDistinctValues(unknownRowCount, unknownRowCount).getVariableStatistics(VARIABLE).getHistogram(), Optional.empty()); // check when rows are available histograms are added properly. - ConnectorHistogram addedSameRange = DisjointRangeDomainHistogram.addDisjunction(unknownNullsFraction.getVariableStatistics(VARIABLE).getHistogram().get(), zeroToTen); + ConnectorHistogram addedSameRange = DisjointRangeDomainHistogram.addDisjunction(unknownNullsFraction.getVariableStatistics(VARIABLE).getHistogram().get(), zeroToTen.toPrestoRange()); assertAddStatsHistogram(unknownNullsFraction, unknownNullsFraction, calculator::addStatsAndSumDistinctValues, addedSameRange); assertAddStatsHistogram(unknownNullsFraction, unknownNullsFraction, calculator::addStatsAndCollapseDistinctValues, addedSameRange); assertAddStatsHistogram(unknownNullsFraction, unknownNullsFraction, calculator::addStatsAndMaxDistinctValues, addedSameRange); assertAddStatsHistogram(unknownNullsFraction, unknownNullsFraction, calculator::addStatsAndIntersect, addedSameRange); // check when only a sub-range is added, that the histogram still represents the full range - ConnectorHistogram fullRangeFirst = DisjointRangeDomainHistogram.addDisjunction(first.getVariableStatistics(VARIABLE).getHistogram().get(), zeroToTen); - ConnectorHistogram intersectedRangeSecond = DisjointRangeDomainHistogram.addConjunction(first.getVariableStatistics(VARIABLE).getHistogram().get(), zeroToFive); + ConnectorHistogram fullRangeFirst = DisjointRangeDomainHistogram.addDisjunction(first.getVariableStatistics(VARIABLE).getHistogram().get(), zeroToTen.toPrestoRange()); + ConnectorHistogram intersectedRangeSecond = DisjointRangeDomainHistogram.addConjunction(first.getVariableStatistics(VARIABLE).getHistogram().get(), zeroToFive.toPrestoRange()); assertAddStatsHistogram(first, second, calculator::addStatsAndSumDistinctValues, fullRangeFirst); assertAddStatsHistogram(first, second, calculator::addStatsAndCollapseDistinctValues, fullRangeFirst); assertAddStatsHistogram(first, second, calculator::addStatsAndMaxDistinctValues, fullRangeFirst); assertAddStatsHistogram(first, second, calculator::addStatsAndIntersect, intersectedRangeSecond); // check when two ranges overlap, the new stats span both ranges - ConnectorHistogram fullRangeSecondThird = DisjointRangeDomainHistogram.addDisjunction(second.getVariableStatistics(VARIABLE).getHistogram().get(), fiveToTen); - ConnectorHistogram intersectedRangeSecondThird = DisjointRangeDomainHistogram.addConjunction(second.getVariableStatistics(VARIABLE).getHistogram().get(), fiveToTen); + ConnectorHistogram fullRangeSecondThird = DisjointRangeDomainHistogram.addDisjunction(second.getVariableStatistics(VARIABLE).getHistogram().get(), fiveToTen.toPrestoRange()); + ConnectorHistogram intersectedRangeSecondThird = DisjointRangeDomainHistogram.addConjunction(second.getVariableStatistics(VARIABLE).getHistogram().get(), fiveToTen.toPrestoRange()); assertAddStatsHistogram(second, third, calculator::addStatsAndSumDistinctValues, fullRangeSecondThird); assertAddStatsHistogram(second, third, calculator::addStatsAndCollapseDistinctValues, fullRangeSecondThird); assertAddStatsHistogram(second, third, calculator::addStatsAndMaxDistinctValues, fullRangeSecondThird); assertAddStatsHistogram(second, third, calculator::addStatsAndIntersect, intersectedRangeSecondThird); // check when two ranges partially overlap, the addition/intersection is applied correctly - ConnectorHistogram fullRangeThirdFourth = DisjointRangeDomainHistogram.addDisjunction(third.getVariableStatistics(VARIABLE).getHistogram().get(), threeToSeven); - ConnectorHistogram intersectedRangeThirdFourth = DisjointRangeDomainHistogram.addConjunction(third.getVariableStatistics(VARIABLE).getHistogram().get(), threeToSeven); + ConnectorHistogram fullRangeThirdFourth = DisjointRangeDomainHistogram.addDisjunction(third.getVariableStatistics(VARIABLE).getHistogram().get(), threeToSeven.toPrestoRange()); + ConnectorHistogram intersectedRangeThirdFourth = DisjointRangeDomainHistogram.addConjunction(third.getVariableStatistics(VARIABLE).getHistogram().get(), threeToSeven.toPrestoRange()); assertAddStatsHistogram(third, fourth, calculator::addStatsAndSumDistinctValues, fullRangeThirdFourth); assertAddStatsHistogram(third, fourth, calculator::addStatsAndCollapseDistinctValues, fullRangeThirdFourth); assertAddStatsHistogram(third, fourth, calculator::addStatsAndMaxDistinctValues, fullRangeThirdFourth); @@ -419,7 +421,7 @@ private static PlanNodeStatsEstimate statistics(double rowCount, double totalSiz .setNullsFraction(nullsFraction) .setAverageRowSize(averageRowSize) .setStatisticsRange(range) - .setHistogram(Optional.of(DisjointRangeDomainHistogram.addConjunction(new UniformDistributionHistogram(range.getLow(), range.getHigh()), range))) + .setHistogram(Optional.of(DisjointRangeDomainHistogram.addConjunction(new UniformDistributionHistogram(range.getLow(), range.getHigh()), range.toPrestoRange()))) .build()) .build(); } diff --git a/presto-main/src/test/java/com/facebook/presto/cost/TestVariableStatsEstimate.java b/presto-main/src/test/java/com/facebook/presto/cost/TestVariableStatsEstimate.java index b26665eb30af..b0f364d1f34e 100644 --- a/presto-main/src/test/java/com/facebook/presto/cost/TestVariableStatsEstimate.java +++ b/presto-main/src/test/java/com/facebook/presto/cost/TestVariableStatsEstimate.java @@ -15,7 +15,7 @@ package com.facebook.presto.cost; import com.facebook.airlift.json.JsonCodec; -import com.google.common.collect.Range; +import com.facebook.presto.spi.statistics.UniformDistributionHistogram; import org.testng.annotations.Test; import java.util.Optional; @@ -33,7 +33,7 @@ public void testSkipHistogramSerialization() VariableStatsEstimate estimate = VariableStatsEstimate.builder() .setAverageRowSize(100) .setDistinctValuesCount(100) - .setStatisticsRange(StatisticRange.fromRange(Range.open(1.0d, 2.0d))) + .setStatisticsRange(new StatisticRange(55, 65, 100)) .setHistogram(Optional.of(new UniformDistributionHistogram(55, 65))) .setNullsFraction(0.1) .build(); diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/ApproximateStatsOutputRowCountMatcher.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/ApproximateStatsOutputRowCountMatcher.java new file mode 100644 index 000000000000..720adc0a0bf5 --- /dev/null +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/ApproximateStatsOutputRowCountMatcher.java @@ -0,0 +1,54 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.sql.planner.assertions; + +import com.facebook.presto.Session; +import com.facebook.presto.cost.StatsProvider; +import com.facebook.presto.metadata.Metadata; +import com.facebook.presto.spi.plan.PlanNode; + +import static com.google.common.base.Verify.verify; + +public class ApproximateStatsOutputRowCountMatcher + implements Matcher +{ + private final double expectedOutputRowCount; + private final double error; + + ApproximateStatsOutputRowCountMatcher(double expectedOutputRowCount, double error) + { + verify(error >= 0.0, "error must be >= 0.0"); + verify(expectedOutputRowCount >= 0.0, "expectedOutputRowCount must be >= 0.0"); + this.expectedOutputRowCount = expectedOutputRowCount; + this.error = error; + } + + @Override + public boolean shapeMatches(PlanNode node) + { + return true; + } + + @Override + public MatchResult detailMatches(PlanNode node, StatsProvider stats, Session session, Metadata metadata, SymbolAliases symbolAliases) + { + return new MatchResult(Math.abs(stats.getStats(node).getOutputRowCount() - expectedOutputRowCount) < error); + } + + @Override + public String toString() + { + return "approximateExpectedOutputRowCount(" + expectedOutputRowCount + ", " + error + ")"; + } +} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanMatchPattern.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanMatchPattern.java index e059c42e89a9..d06a36ae8127 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanMatchPattern.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanMatchPattern.java @@ -815,6 +815,12 @@ public PlanMatchPattern withOutputRowCount(boolean exactMatch, String expectedSo return this; } + public PlanMatchPattern withApproximateOutputRowCount(double expectedOutputRowCount, double error) + { + matchers.add(new ApproximateStatsOutputRowCountMatcher(expectedOutputRowCount, error)); + return this; + } + public PlanMatchPattern withOutputSize(double expectedOutputSize) { matchers.add(new StatsOutputSizeMatcher(expectedOutputSize)); diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index c1099d4e8597..9a06db72ca29 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -126,5 +126,11 @@ assertj-core test + + + org.apache.commons + commons-math3 + test + diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/statistics/ColumnStatistics.java b/presto-spi/src/main/java/com/facebook/presto/spi/statistics/ColumnStatistics.java index 97b1ac7fd4c1..255c72835099 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/statistics/ColumnStatistics.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/statistics/ColumnStatistics.java @@ -29,6 +29,8 @@ public final class ColumnStatistics private static final long COLUMN_STATISTICS_SIZE = ClassLayout.parseClass(ColumnStatistics.class).instanceSize(); private static final long OPTION_SIZE = ClassLayout.parseClass(Optional.class).instanceSize(); + public static final double INFINITE_TO_FINITE_RANGE_INTERSECT_OVERLAP_HEURISTIC_FACTOR = 0.25; + public static final double INFINITE_TO_INFINITE_RANGE_INTERSECT_OVERLAP_HEURISTIC_FACTOR = 0.5; private static final ColumnStatistics EMPTY = new ColumnStatistics(Estimate.unknown(), Estimate.unknown(), Estimate.unknown(), Optional.empty(), Optional.empty()); private final Estimate nullsFraction; @@ -224,6 +226,11 @@ public Builder setHistogram(Optional histogram) return this; } + public Optional getHistogram() + { + return histogram; + } + public Builder mergeWith(Builder other) { if (nullsFraction.isUnknown()) { diff --git a/presto-main/src/main/java/com/facebook/presto/cost/DisjointRangeDomainHistogram.java b/presto-spi/src/main/java/com/facebook/presto/spi/statistics/DisjointRangeDomainHistogram.java similarity index 74% rename from presto-main/src/main/java/com/facebook/presto/cost/DisjointRangeDomainHistogram.java rename to presto-spi/src/main/java/com/facebook/presto/spi/statistics/DisjointRangeDomainHistogram.java index 553cf84d07bf..2d5f44550b2d 100644 --- a/presto-main/src/main/java/com/facebook/presto/cost/DisjointRangeDomainHistogram.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/statistics/DisjointRangeDomainHistogram.java @@ -12,36 +12,33 @@ * limitations under the License. */ -package com.facebook.presto.cost; +package com.facebook.presto.spi.statistics; -import com.facebook.presto.spi.statistics.ConnectorHistogram; -import com.facebook.presto.spi.statistics.Estimate; +import com.facebook.presto.common.predicate.Marker; +import com.facebook.presto.common.predicate.Range; +import com.facebook.presto.common.predicate.SortedRangeSet; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Suppliers; -import com.google.common.collect.BoundType; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Range; -import com.google.common.collect.RangeSet; -import com.google.common.collect.TreeRangeSet; import org.openjdk.jol.info.ClassLayout; -import java.util.Collection; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; import java.util.NoSuchElementException; import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.function.Supplier; -import static com.facebook.presto.cost.HistogramCalculator.calculateFilterFactor; -import static com.facebook.presto.util.MoreMath.max; -import static com.facebook.presto.util.MoreMath.min; -import static com.google.common.base.MoreObjects.toStringHelper; -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.collect.ImmutableSet.toImmutableSet; +import static com.facebook.presto.common.Utils.checkArgument; +import static com.facebook.presto.common.Utils.memoizedSupplier; +import static com.facebook.presto.common.type.DoubleType.DOUBLE; import static java.lang.Double.NEGATIVE_INFINITY; +import static java.lang.Double.NaN; import static java.lang.Double.POSITIVE_INFINITY; import static java.lang.Double.isFinite; +import static java.lang.Math.max; +import static java.lang.Math.min; import static java.util.Objects.hash; import static java.util.Objects.requireNonNull; @@ -75,31 +72,27 @@ public class DisjointRangeDomainHistogram private final ConnectorHistogram source; // use RangeSet as the internal representation of the ranges, but the constructor arguments // use StatisticRange to support serialization and deserialization. - private final Supplier> rangeSet; - private final Set> ranges; + private final Supplier rangeSet; + private final Set ranges; @JsonCreator - public DisjointRangeDomainHistogram(@JsonProperty("source") ConnectorHistogram source, @JsonProperty("ranges") Collection ranges) - { - this(source, ranges.stream().map(StatisticRange::toRange).collect(toImmutableSet())); - } - - public DisjointRangeDomainHistogram(ConnectorHistogram source, Set> ranges) + public DisjointRangeDomainHistogram(ConnectorHistogram source, Set ranges) { this.source = requireNonNull(source, "source is null"); this.ranges = requireNonNull(ranges, "ranges is null"); - this.rangeSet = Suppliers.memoize(() -> { - RangeSet rangeSet = TreeRangeSet.create(); - rangeSet.addAll(ranges); + this.rangeSet = memoizedSupplier(() -> { + SortedRangeSet rangeSet = SortedRangeSet.copyOf(DOUBLE, new ArrayList<>(ranges)); return rangeSet.subRangeSet(getSourceSpan(this.source)); }); } - private static Range getSourceSpan(ConnectorHistogram source) + private static Range getSourceSpan(ConnectorHistogram source) { - return Range.closed( + return Range.range(DOUBLE, source.inverseCumulativeProbability(0.0).orElse(() -> NEGATIVE_INFINITY), - source.inverseCumulativeProbability(1.0).orElse(() -> POSITIVE_INFINITY)); + true, + source.inverseCumulativeProbability(1.0).orElse(() -> POSITIVE_INFINITY), + true); } @JsonProperty @@ -109,14 +102,14 @@ public ConnectorHistogram getSource() } @JsonProperty - public Set getRanges() + public SortedRangeSet getRanges() { - return rangeSet.get().asRanges().stream().map(StatisticRange::fromRange).collect(toImmutableSet()); + return rangeSet.get(); } public DisjointRangeDomainHistogram(ConnectorHistogram source) { - this(source, ImmutableSet.>of()); + this(source, Collections.emptySet()); } @Override @@ -130,17 +123,22 @@ public Estimate cumulativeProbability(double value, boolean inclusive) if (Double.isNaN(value)) { return Estimate.unknown(); } - Optional> optionalSpan = getSpan(); + Optional optionalSpan = getSpan(); if (!optionalSpan.isPresent()) { return Estimate.of(0.0); } - Range span = optionalSpan.get(); - if (value <= span.lowerEndpoint()) { + Range span = optionalSpan.get(); + if (value <= span.getLowValue().map(Double.class::cast) + .orElse(NEGATIVE_INFINITY)) { return Estimate.of(0.0); } - Range input = Range.range(span.lowerEndpoint(), span.lowerBoundType(), value, inclusive ? BoundType.CLOSED : BoundType.OPEN); + Range input = Range.range(DOUBLE, + span.getLowValue().map(Double.class::cast).orElse(NEGATIVE_INFINITY), + span.getLow().getBound() == Marker.Bound.EXACTLY, + value, + inclusive); Estimate fullSetOverlap = calculateRangeSetOverlap(rangeSet.get()); - RangeSet spanned = rangeSet.get().subRangeSet(input); + SortedRangeSet spanned = rangeSet.get().subRangeSet(input); Estimate spannedOverlap = calculateRangeSetOverlap(spanned); return spannedOverlap.flatMap(spannedProbability -> @@ -152,11 +150,11 @@ public Estimate cumulativeProbability(double value, boolean inclusive) })); } - private Estimate calculateRangeSetOverlap(RangeSet ranges) + private Estimate calculateRangeSetOverlap(SortedRangeSet ranges) { // we require knowing bounds on all ranges double cumulativeTotal = 0.0; - for (Range range : ranges.asRanges()) { + for (Range range : ranges.getOrderedRanges()) { Estimate rangeProbability = getRangeProbability(range); if (rangeProbability.isUnknown()) { return Estimate.unknown(); @@ -173,9 +171,9 @@ private Estimate calculateRangeSetOverlap(RangeSet ranges) * @param range the range over the source domain * @return estimate of the total probability the range covers in the source */ - private Estimate getRangeProbability(Range range) + private Estimate getRangeProbability(Range range) { - return calculateFilterFactor(StatisticRange.fromRange(range), source, Estimate.unknown(), false); + return HistogramCalculator.calculateFilterFactor(range, NaN, source, Estimate.unknown(), false); } @Override @@ -189,17 +187,19 @@ public Estimate inverseCumulativeProbability(double percentile) // rangedPercentile = percentile - percentileLow // // percentileLow + (rangedPercentile * rangePercentileLength) - Optional> optionalSpan = getSpan(); + Optional optionalSpan = getSpan(); if (!optionalSpan.isPresent()) { return Estimate.unknown(); } - Range span = optionalSpan.get(); - if (percentile == 0.0 && isFinite(span.lowerEndpoint())) { - return source.inverseCumulativeProbability(0.0).map(sourceMin -> max(span.lowerEndpoint(), sourceMin)); + Range span = optionalSpan.get(); + double lower = span.getLowValue().map(Double.class::cast).orElse(NEGATIVE_INFINITY); + double upper = span.getHighValue().map(Double.class::cast).orElse(POSITIVE_INFINITY); + if (percentile == 0.0 && isFinite(lower)) { + return source.inverseCumulativeProbability(0.0).map(sourceMin -> max(lower, sourceMin)); } - if (percentile == 1.0 && isFinite(span.upperEndpoint())) { - return source.inverseCumulativeProbability(1.0).map(sourceMax -> min(span.upperEndpoint(), sourceMax)); + if (percentile == 1.0 && isFinite(upper)) { + return source.inverseCumulativeProbability(1.0).map(sourceMax -> min(upper, sourceMax)); } Estimate totalCumulativeEstimate = calculateRangeSetOverlap(rangeSet.get()); @@ -213,9 +213,9 @@ public Estimate inverseCumulativeProbability(double percentile) } double cumulativeProbabilityNewDomain = 0.0; double lastRangeEstimateSourceDomain = 0.0; - Range currentRange = null; + Range currentRange = null; // find the range where the percentile falls - for (Range range : rangeSet.get().asRanges()) { + for (Range range : rangeSet.get().getOrderedRanges()) { Estimate rangeEstimate = getRangeProbability(range); if (rangeEstimate.isUnknown()) { return Estimate.unknown(); @@ -231,7 +231,8 @@ public Estimate inverseCumulativeProbability(double percentile) // no ranges to iterate over. Did a constraint cut the entire domain of values? return Estimate.unknown(); } - Estimate rangeLeftSourceEstimate = source.cumulativeProbability(currentRange.lowerEndpoint(), currentRange.lowerBoundType() == BoundType.OPEN); + Double currentLow = currentRange.getLowValue().map(Double.class::cast).orElse(NEGATIVE_INFINITY); + Estimate rangeLeftSourceEstimate = source.cumulativeProbability(currentLow, !currentRange.isLowInclusive()); if (rangeLeftSourceEstimate.isUnknown()) { return Estimate.unknown(); } @@ -250,12 +251,10 @@ public Estimate inverseCumulativeProbability(double percentile) * @param other the new range to add to the set. * @return a new {@link DisjointRangeDomainHistogram} */ - public DisjointRangeDomainHistogram addDisjunction(StatisticRange other) + public DisjointRangeDomainHistogram addDisjunction(Range other) { - Set> ranges = ImmutableSet.>builder() - .addAll(this.ranges) - .add(other.toRange()) - .build(); + Set ranges = new HashSet<>(this.ranges); + ranges.add(other); return new DisjointRangeDomainHistogram(source, ranges); } @@ -266,9 +265,9 @@ public DisjointRangeDomainHistogram addDisjunction(StatisticRange other) * @param other the range that should enclose the set. * @return a new {@link DisjointRangeDomainHistogram} where */ - public DisjointRangeDomainHistogram addConjunction(StatisticRange other) + public DisjointRangeDomainHistogram addConjunction(Range other) { - return new DisjointRangeDomainHistogram(source, rangeSet.get().subRangeSet(other.toRange()).asRanges()); + return new DisjointRangeDomainHistogram(source, new HashSet<>(rangeSet.get().subRangeSet(other).getOrderedRanges())); } /** @@ -287,17 +286,17 @@ public DisjointRangeDomainHistogram addConjunction(StatisticRange other) * @param range the range representing the conjunction to add * @return a new histogram with the conjunction applied. */ - public static ConnectorHistogram addDisjunction(ConnectorHistogram histogram, StatisticRange range) + public static ConnectorHistogram addDisjunction(ConnectorHistogram histogram, Range range) { if (histogram instanceof DisjointRangeDomainHistogram) { return ((DisjointRangeDomainHistogram) histogram).addDisjunction(range); } - return new DisjointRangeDomainHistogram(histogram, ImmutableSet.of(range.toRange())); + return new DisjointRangeDomainHistogram(histogram, Collections.singleton(range)); } /** - * Similar to {@link #addDisjunction(ConnectorHistogram, StatisticRange)} this method constrains + * Similar to {@link #addDisjunction(ConnectorHistogram, Range)} this method constrains * the entire domain such that all ranges in the set intersect with the given range * argument to this method. *
@@ -308,22 +307,24 @@ public static ConnectorHistogram addDisjunction(ConnectorHistogram histogram, St * @param range the range of values that the entire histogram's domain must fall within * @return a histogram with the new range constraint */ - public static ConnectorHistogram addConjunction(ConnectorHistogram histogram, StatisticRange range) + public static ConnectorHistogram addConjunction(ConnectorHistogram histogram, Range range) { if (histogram instanceof DisjointRangeDomainHistogram) { return ((DisjointRangeDomainHistogram) histogram).addConjunction(range); } - return new DisjointRangeDomainHistogram(histogram, ImmutableSet.of(range.toRange())); + return new DisjointRangeDomainHistogram(histogram, Collections.singleton(range)); } /** * @return the span if it exists, empty otherwise */ - private Optional> getSpan() + private Optional getSpan() { try { - return Optional.of(rangeSet.get().span()); + return Optional.of(rangeSet.get()) + .filter(set -> !set.isNone()) // prevent exception + .map(SortedRangeSet::getSpan); } catch (NoSuchElementException e) { return Optional.empty(); @@ -333,10 +334,10 @@ private Optional> getSpan() @Override public String toString() { - return toStringHelper(this) - .add("source", this.source) - .add("rangeSet", this.rangeSet) - .toString(); + return "DisjointRangeDomainHistogram{" + + "source=" + this.source + ", " + + ", rangeSet" + this.rangeSet.get() + + "}"; } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/cost/HistogramCalculator.java b/presto-spi/src/main/java/com/facebook/presto/spi/statistics/HistogramCalculator.java similarity index 68% rename from presto-main/src/main/java/com/facebook/presto/cost/HistogramCalculator.java rename to presto-spi/src/main/java/com/facebook/presto/spi/statistics/HistogramCalculator.java index 12525b6120cc..8db65dbc69ff 100644 --- a/presto-main/src/main/java/com/facebook/presto/cost/HistogramCalculator.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/statistics/HistogramCalculator.java @@ -12,12 +12,14 @@ * limitations under the License. */ -package com.facebook.presto.cost; +package com.facebook.presto.spi.statistics; -import com.facebook.presto.spi.statistics.ConnectorHistogram; -import com.facebook.presto.spi.statistics.Estimate; -import com.google.common.math.DoubleMath; +import com.facebook.presto.common.predicate.Range; +import static com.facebook.presto.spi.statistics.ColumnStatistics.INFINITE_TO_FINITE_RANGE_INTERSECT_OVERLAP_HEURISTIC_FACTOR; +import static com.facebook.presto.spi.statistics.ColumnStatistics.INFINITE_TO_INFINITE_RANGE_INTERSECT_OVERLAP_HEURISTIC_FACTOR; +import static java.lang.Double.NEGATIVE_INFINITY; +import static java.lang.Double.POSITIVE_INFINITY; import static java.lang.Double.isFinite; import static java.lang.Double.isNaN; import static java.lang.Math.min; @@ -43,16 +45,19 @@ private HistogramCalculator() * heuristic would have been used * @return an estimate, x, where 0.0 <= x <= 1.0. */ - public static Estimate calculateFilterFactor(StatisticRange range, ConnectorHistogram histogram, Estimate totalDistinctValues, boolean useHeuristics) + public static Estimate calculateFilterFactor(Range range, double rangeDistinctValues, ConnectorHistogram histogram, Estimate totalDistinctValues, boolean useHeuristics) { - boolean openHigh = range.getOpenHigh(); - boolean openLow = range.getOpenLow(); + boolean openHigh = !range.isHighInclusive(); + boolean openLow = !range.isLowInclusive(); Estimate min = histogram.inverseCumulativeProbability(0.0); Estimate max = histogram.inverseCumulativeProbability(1.0); + double rangeLow = range.getLowValue().map(Double.class::cast).orElse(NEGATIVE_INFINITY); + double rangeHigh = range.getHighValue().map(Double.class::cast).orElse(POSITIVE_INFINITY); + double rangeLength = rangeHigh - rangeLow; // range is either above or below histogram - if ((!max.isUnknown() && (openHigh ? max.getValue() <= range.getLow() : max.getValue() < range.getLow())) - || (!min.isUnknown() && (openLow ? min.getValue() >= range.getHigh() : min.getValue() > range.getHigh()))) { + if ((!max.isUnknown() && (openHigh ? max.getValue() <= rangeLow : max.getValue() < rangeLow)) + || (!min.isUnknown() && (openLow ? min.getValue() >= rangeHigh : min.getValue() > rangeHigh))) { return Estimate.of(0.0); } @@ -63,14 +68,14 @@ public static Estimate calculateFilterFactor(StatisticRange range, ConnectorHist return Estimate.unknown(); } - if (range.length() == 0.0) { + if (rangeLength == 0.0) { return totalDistinctValues.map(distinct -> 1.0 / distinct); } - if (isFinite(range.length())) { - return Estimate.of(StatisticRange.INFINITE_TO_FINITE_RANGE_INTERSECT_OVERLAP_HEURISTIC_FACTOR); + if (isFinite(rangeLength)) { + return Estimate.of(INFINITE_TO_FINITE_RANGE_INTERSECT_OVERLAP_HEURISTIC_FACTOR); } - return Estimate.of(StatisticRange.INFINITE_TO_INFINITE_RANGE_INTERSECT_OVERLAP_HEURISTIC_FACTOR); + return Estimate.of(INFINITE_TO_INFINITE_RANGE_INTERSECT_OVERLAP_HEURISTIC_FACTOR); } // we know the bounds are both known, so calculate the percentile for each bound @@ -82,8 +87,8 @@ public static Estimate calculateFilterFactor(StatisticRange range, ConnectorHist // thus for the "lowPercentile" calculation we should pass "false" to be non-inclusive // (same as openness) however, on the high-end we want the inclusivity to be the opposite // of the openness since if it's open, we _don't_ want to include the bound. - Estimate lowPercentile = histogram.cumulativeProbability(range.getLow(), openLow); - Estimate highPercentile = histogram.cumulativeProbability(range.getHigh(), !openHigh); + Estimate lowPercentile = histogram.cumulativeProbability(rangeLow, openLow); + Estimate highPercentile = histogram.cumulativeProbability(rangeHigh, !openHigh); // both bounds are probably infinity, use the infinite-infinite heuristic if (lowPercentile.isUnknown() || highPercentile.isUnknown()) { @@ -91,26 +96,26 @@ public static Estimate calculateFilterFactor(StatisticRange range, ConnectorHist return Estimate.unknown(); } // in the case the histogram has no values - if (totalDistinctValues.equals(Estimate.zero()) || range.getDistinctValuesCount() == 0.0) { + if (totalDistinctValues.equals(Estimate.zero()) || rangeDistinctValues == 0.0) { return Estimate.of(0.0); } // in the case only one is unknown if (((lowPercentile.isUnknown() && !highPercentile.isUnknown()) || (!lowPercentile.isUnknown() && highPercentile.isUnknown())) && - isFinite(range.length())) { - return Estimate.of(StatisticRange.INFINITE_TO_FINITE_RANGE_INTERSECT_OVERLAP_HEURISTIC_FACTOR); + isFinite(rangeLength)) { + return Estimate.of(INFINITE_TO_FINITE_RANGE_INTERSECT_OVERLAP_HEURISTIC_FACTOR); } - if (range.length() == 0.0) { + if (rangeLength == 0.0) { return totalDistinctValues.map(distinct -> 1.0 / distinct); } - if (!isNaN(range.getDistinctValuesCount())) { - return totalDistinctValues.map(distinct -> min(1.0, range.getDistinctValuesCount() / distinct)); + if (!isNaN(rangeDistinctValues)) { + return totalDistinctValues.map(distinct -> min(1.0, rangeDistinctValues / distinct)); } - return Estimate.of(StatisticRange.INFINITE_TO_INFINITE_RANGE_INTERSECT_OVERLAP_HEURISTIC_FACTOR); + return Estimate.of(INFINITE_TO_INFINITE_RANGE_INTERSECT_OVERLAP_HEURISTIC_FACTOR); } // in the case the range is a single value, this can occur if the input @@ -134,15 +139,23 @@ public static Estimate calculateFilterFactor(StatisticRange range, ConnectorHist } return totalDistinctValues.flatMap(totalDistinct -> { - if (DoubleMath.fuzzyEquals(totalDistinct, 0.0, 1E-6)) { + if (fuzzyEquals(totalDistinct, 0.0, 1E-6)) { return Estimate.of(1.0); } - return Estimate.of(min(1.0, range.getDistinctValuesCount() / totalDistinct)); + return Estimate.of(min(1.0, rangeDistinctValues / totalDistinct)); }) - // in the case totalDistinct is NaN or 0 - .or(() -> Estimate.of(StatisticRange.INFINITE_TO_INFINITE_RANGE_INTERSECT_OVERLAP_HEURISTIC_FACTOR)); + // in the case totalDistinct is NaN or 0 + .or(() -> Estimate.of(INFINITE_TO_INFINITE_RANGE_INTERSECT_OVERLAP_HEURISTIC_FACTOR)); } return lowPercentile.flatMap(lowPercent -> highPercentile.map(highPercent -> highPercent - lowPercent)); } + + private static boolean fuzzyEquals(double a, double b, double tolerance) + { + return Math.copySign(a - b, 1.0) <= tolerance + // copySign(x, 1.0) is a branch-free version of abs(x), but with different NaN semantics + || (a == b) // needed to ensure that infinities equal themselves + || (Double.isNaN(a) && Double.isNaN(b)); + } } diff --git a/presto-main/src/main/java/com/facebook/presto/cost/UniformDistributionHistogram.java b/presto-spi/src/main/java/com/facebook/presto/spi/statistics/UniformDistributionHistogram.java similarity index 85% rename from presto-main/src/main/java/com/facebook/presto/cost/UniformDistributionHistogram.java rename to presto-spi/src/main/java/com/facebook/presto/spi/statistics/UniformDistributionHistogram.java index c9cb6bfd1952..a1a95aa974ec 100644 --- a/presto-main/src/main/java/com/facebook/presto/cost/UniformDistributionHistogram.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/statistics/UniformDistributionHistogram.java @@ -12,17 +12,13 @@ * limitations under the License. */ -package com.facebook.presto.cost; +package com.facebook.presto.spi.statistics; -import com.facebook.presto.spi.statistics.ConnectorHistogram; -import com.facebook.presto.spi.statistics.Estimate; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.openjdk.jol.info.ClassLayout; -import static com.google.common.base.MoreObjects.toStringHelper; -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Verify.verify; +import static com.facebook.presto.common.Utils.checkArgument; import static java.lang.Double.isInfinite; import static java.lang.Double.isNaN; import static java.lang.Math.max; @@ -48,7 +44,7 @@ public UniformDistributionHistogram( @JsonProperty("lowValue") double lowValue, @JsonProperty("highValue") double highValue) { - verify(isNaN(lowValue) || isNaN(highValue) || (lowValue <= highValue), "lowValue must be <= highValue"); + checkArgument(isNaN(lowValue) || isNaN(highValue) || (lowValue <= highValue), "lowValue must be <= highValue"); this.lowValue = lowValue; this.highValue = highValue; } @@ -122,10 +118,10 @@ public long getEstimatedSize() @Override public String toString() { - return toStringHelper(this) - .add("lowValue", lowValue) - .add("highValue", highValue) - .toString(); + return "UniformDistributionHistogram{" + + "lowValue=" + lowValue + + ", highValue=" + highValue + + "}"; } @Override @@ -151,6 +147,6 @@ public int hashCode() private static boolean equalsOrBothNaN(Double first, Double second) { - return first.equals(second) || (Double.isNaN(first) && Double.isNaN(second)); + return first.equals(second) || (isNaN(first) && isNaN(second)); } } diff --git a/presto-main/src/test/java/com/facebook/presto/cost/TestDisjointRangeDomainHistogram.java b/presto-spi/src/test/java/com/facebook/presto/spi/statistics/TestDisjointRangeDomainHistogram.java similarity index 81% rename from presto-main/src/test/java/com/facebook/presto/cost/TestDisjointRangeDomainHistogram.java rename to presto-spi/src/test/java/com/facebook/presto/spi/statistics/TestDisjointRangeDomainHistogram.java index 0c11e729b1fe..6939becf267c 100644 --- a/presto-main/src/test/java/com/facebook/presto/cost/TestDisjointRangeDomainHistogram.java +++ b/presto-spi/src/test/java/com/facebook/presto/spi/statistics/TestDisjointRangeDomainHistogram.java @@ -12,20 +12,20 @@ * limitations under the License. */ -package com.facebook.presto.cost; +package com.facebook.presto.spi.statistics; -import com.facebook.presto.spi.statistics.ConnectorHistogram; -import com.facebook.presto.spi.statistics.Estimate; import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Range; import org.apache.commons.math3.distribution.NormalDistribution; import org.apache.commons.math3.distribution.RealDistribution; import org.apache.commons.math3.distribution.UniformRealDistribution; import org.testng.annotations.Test; import java.util.List; -import java.util.stream.Collectors; +import static com.facebook.presto.common.predicate.Range.greaterThanOrEqual; +import static com.facebook.presto.common.predicate.Range.lessThanOrEqual; +import static com.facebook.presto.common.predicate.Range.range; +import static com.facebook.presto.common.type.DoubleType.DOUBLE; import static org.testng.Assert.assertEquals; public class TestDisjointRangeDomainHistogram @@ -39,9 +39,9 @@ public void testBasicDisjointRanges() { ConnectorHistogram source = new UniformDistributionHistogram(0, 100); ConnectorHistogram constrained = DisjointRangeDomainHistogram - .addDisjunction(source, StatisticRange.fromRange(Range.open(0d, 25d))); + .addDisjunction(source, rangeOpen(0d, 25d)); constrained = DisjointRangeDomainHistogram - .addDisjunction(constrained, StatisticRange.fromRange(Range.open(75d, 100d))); + .addDisjunction(constrained, rangeOpen(75d, 100d)); assertEquals(constrained.inverseCumulativeProbability(0.75).getValue(), 87.5); assertEquals(constrained.inverseCumulativeProbability(0.0).getValue(), 0.0); assertEquals(constrained.inverseCumulativeProbability(1.0).getValue(), 100); @@ -59,7 +59,7 @@ public void testSingleDisjointRange() // no overlap, left bound ConnectorHistogram constrained = DisjointRangeDomainHistogram - .addDisjunction(source, StatisticRange.fromRange(Range.open(-10d, -5d))); + .addDisjunction(source, rangeOpen(-10d, -5d)); for (int i = -11; i < 12; i++) { assertEquals(constrained.cumulativeProbability(i, true).getValue(), 0.0, 1E-8); assertEquals(constrained.cumulativeProbability(i, false).getValue(), 0.0, 1E-8); @@ -68,7 +68,7 @@ public void testSingleDisjointRange() assertEquals(constrained.inverseCumulativeProbability(1.0), Estimate.unknown()); // partial overlap left bound - constrained = new DisjointRangeDomainHistogram(source, ImmutableSet.of(Range.open(-2d, 2d))); + constrained = new DisjointRangeDomainHistogram(source, ImmutableSet.of(rangeOpen(-2d, 2d))); assertEquals(constrained.cumulativeProbability(-3, false).getValue(), 0.0, 1E-8); assertEquals(constrained.cumulativeProbability(-1, false).getValue(), 0.0, 1E-8); assertEquals(constrained.cumulativeProbability(0, false).getValue(), 0.0, 1E-8); @@ -82,7 +82,7 @@ public void testSingleDisjointRange() assertEquals(constrained.inverseCumulativeProbability(1.0).getValue(), 2d, 1E-8); //full overlap - constrained = new DisjointRangeDomainHistogram(source, ImmutableSet.of(Range.open(3d, 4d))); + constrained = new DisjointRangeDomainHistogram(source, ImmutableSet.of(rangeOpen(3d, 4d))); assertEquals(constrained.cumulativeProbability(-3, false).getValue(), 0.0, 1E-8); assertEquals(constrained.cumulativeProbability(0, false).getValue(), 0.0, 1E-8); assertEquals(constrained.cumulativeProbability(1, false).getValue(), 0.0, 1E-8); @@ -96,7 +96,7 @@ public void testSingleDisjointRange() assertEquals(constrained.inverseCumulativeProbability(1.0).getValue(), 4d, 1E-8); //right side overlap - constrained = new DisjointRangeDomainHistogram(source, ImmutableSet.of(Range.open(8d, 12d))); + constrained = new DisjointRangeDomainHistogram(source, ImmutableSet.of(rangeOpen(8d, 12d))); assertEquals(constrained.cumulativeProbability(-3, false).getValue(), 0.0, 1E-8); assertEquals(constrained.cumulativeProbability(0, false).getValue(), 0.0, 1E-8); assertEquals(constrained.cumulativeProbability(5, false).getValue(), 0.0, 1E-8); @@ -114,7 +114,7 @@ public void testSingleDisjointRange() // no overlap, right bound constrained = DisjointRangeDomainHistogram - .addDisjunction(source, StatisticRange.fromRange(Range.open(15d, 20d))); + .addDisjunction(source, rangeOpen(15d, 20d)); for (int i = 15; i < 20; i++) { assertEquals(constrained.cumulativeProbability(i, true).getValue(), 0.0, 1E-8); assertEquals(constrained.cumulativeProbability(i, false).getValue(), 0.0, 1E-8); @@ -132,8 +132,8 @@ public void testMultipleDisjunction() { StandardNormalHistogram source = new StandardNormalHistogram(); RealDistribution dist = source.getDistribution(); - ConnectorHistogram constrained = disjunction(source, Range.closed(-2d, -1d)); - constrained = disjunction(constrained, Range.closed(1d, 2d)); + ConnectorHistogram constrained = disjunction(source, rangeClosed(-2d, -1d)); + constrained = disjunction(constrained, rangeClosed(1d, 2d)); double rangeLeftProb = dist.cumulativeProbability(-1) - dist.cumulativeProbability(-2); double rangeRightProb = dist.cumulativeProbability(2) - dist.cumulativeProbability(1); double sumRangeProb = rangeLeftProb + rangeRightProb; @@ -156,7 +156,7 @@ public void testNormalDistribution() // standard normal StandardNormalHistogram source = new StandardNormalHistogram(); RealDistribution dist = source.getDistribution(); - ConnectorHistogram constrained = new DisjointRangeDomainHistogram(source, ImmutableSet.of(Range.open(-1d, 1d))); + ConnectorHistogram constrained = new DisjointRangeDomainHistogram(source, ImmutableSet.of(rangeOpen(-1d, 1d))); assertEquals(constrained.cumulativeProbability(-1.0, true).getValue(), 0.0, 1E-8); assertEquals(constrained.cumulativeProbability(0.0, true).getValue(), 0.5, 1E-8); assertEquals(constrained.cumulativeProbability(1.0, true).getValue(), 1.0, 1E-8); @@ -179,16 +179,16 @@ public void testNormalDistribution() public void testAddDisjunction() { ConnectorHistogram source = new UniformDistributionHistogram(0, 100); - DisjointRangeDomainHistogram constrained = disjunction(source, Range.open(-1d, 2d)); - assertEquals(constrained.getRanges().size(), 1); - assertEquals(ranges(constrained).get(0), Range.closedOpen(0d, 2d)); - constrained = disjunction(constrained, Range.open(1d, 10d)); + DisjointRangeDomainHistogram constrained = disjunction(source, rangeOpen(-1d, 2d)); + assertEquals(constrained.getRanges().getOrderedRanges().size(), 1); + assertEquals(ranges(constrained).get(0), range(DOUBLE, 0d, true, 2d, false)); + constrained = disjunction(constrained, rangeOpen(1d, 10d)); assertEquals(ranges(constrained).size(), 1); - assertEquals(ranges(constrained).get(0), Range.closedOpen(0d, 10d)); - constrained = disjunction(constrained, Range.closedOpen(50d, 100d)); + assertEquals(ranges(constrained).get(0), range(DOUBLE, 0d, true, 10d, false)); + constrained = disjunction(constrained, range(DOUBLE, 50d, true, 100d, false)); assertEquals(ranges(constrained).size(), 2); - assertEquals(ranges(constrained).get(0), Range.closedOpen(0d, 10d)); - assertEquals(ranges(constrained).get(1), Range.closedOpen(50d, 100d)); + assertEquals(ranges(constrained).get(0), range(DOUBLE, 0d, true, 10d, false)); + assertEquals(ranges(constrained).get(1), range(DOUBLE, 50d, true, 100d, false)); } /** @@ -198,30 +198,40 @@ public void testAddDisjunction() public void testAddConjunction() { ConnectorHistogram source = new UniformDistributionHistogram(0, 100); - DisjointRangeDomainHistogram constrained = disjunction(source, Range.open(10d, 90d)); - assertEquals(constrained.getRanges().size(), 1); - assertEquals(ranges(constrained).get(0), Range.open(10d, 90d)); - constrained = conjunction(constrained, Range.atMost(50d)); + DisjointRangeDomainHistogram constrained = disjunction(source, rangeOpen(10d, 90d)); + assertEquals(constrained.getRanges().getOrderedRanges().size(), 1); + assertEquals(ranges(constrained).get(0), rangeOpen(10d, 90d)); + constrained = conjunction(constrained, lessThanOrEqual(DOUBLE, 50d)); assertEquals(ranges(constrained).size(), 1); - assertEquals(ranges(constrained).get(0), Range.openClosed(10d, 50d)); - constrained = conjunction(constrained, Range.atLeast(25d)); + assertEquals(ranges(constrained).get(0), range(DOUBLE, 10d, false, 50d, true)); + constrained = conjunction(constrained, greaterThanOrEqual(DOUBLE, 25d)); assertEquals(ranges(constrained).size(), 1); - assertEquals(ranges(constrained).get(0), Range.closed(25d, 50d)); + assertEquals(ranges(constrained).get(0), rangeClosed(25d, 50d)); } - private static DisjointRangeDomainHistogram disjunction(ConnectorHistogram source, Range range) + private static DisjointRangeDomainHistogram disjunction(ConnectorHistogram source, com.facebook.presto.common.predicate.Range range) { - return (DisjointRangeDomainHistogram) DisjointRangeDomainHistogram.addDisjunction(source, StatisticRange.fromRange(range)); + return (DisjointRangeDomainHistogram) DisjointRangeDomainHistogram.addDisjunction(source, range); } - private static DisjointRangeDomainHistogram conjunction(ConnectorHistogram source, Range range) + private static DisjointRangeDomainHistogram conjunction(ConnectorHistogram source, com.facebook.presto.common.predicate.Range range) { - return (DisjointRangeDomainHistogram) DisjointRangeDomainHistogram.addConjunction(source, StatisticRange.fromRange(range)); + return (DisjointRangeDomainHistogram) DisjointRangeDomainHistogram.addConjunction(source, range); } - private static List> ranges(DisjointRangeDomainHistogram hist) + private static List ranges(DisjointRangeDomainHistogram hist) { - return hist.getRanges().stream().map(StatisticRange::toRange).collect(Collectors.toList()); + return hist.getRanges().getOrderedRanges(); + } + + private static com.facebook.presto.common.predicate.Range rangeOpen(double low, double high) + { + return range(DOUBLE, low, false, high, false); + } + + private static com.facebook.presto.common.predicate.Range rangeClosed(double low, double high) + { + return range(DOUBLE, low, true, high, true); } private static class StandardNormalHistogram @@ -269,7 +279,7 @@ ConnectorHistogram createHistogram() return new DisjointRangeDomainHistogram( new UniformDistributionHistogram( distribution.getSupportLowerBound(), distribution.getSupportUpperBound())) - .addDisjunction(new StatisticRange(0.0, 100.0, 0.0)); + .addDisjunction(rangeClosed(0.0, 100.0)); } @Override diff --git a/presto-main/src/test/java/com/facebook/presto/cost/TestHistogram.java b/presto-spi/src/test/java/com/facebook/presto/spi/statistics/TestHistogram.java similarity index 97% rename from presto-main/src/test/java/com/facebook/presto/cost/TestHistogram.java rename to presto-spi/src/test/java/com/facebook/presto/spi/statistics/TestHistogram.java index 26c68b7e5730..341870d138bd 100644 --- a/presto-main/src/test/java/com/facebook/presto/cost/TestHistogram.java +++ b/presto-spi/src/test/java/com/facebook/presto/spi/statistics/TestHistogram.java @@ -12,9 +12,8 @@ * limitations under the License. */ -package com.facebook.presto.cost; +package com.facebook.presto.spi.statistics; -import com.facebook.presto.spi.statistics.ConnectorHistogram; import org.apache.commons.math3.distribution.RealDistribution; import org.testng.annotations.Test; diff --git a/presto-spi/src/test/java/com/facebook/presto/spi/statistics/TestHistogramCalculator.java b/presto-spi/src/test/java/com/facebook/presto/spi/statistics/TestHistogramCalculator.java new file mode 100644 index 000000000000..0632e14247b1 --- /dev/null +++ b/presto-spi/src/test/java/com/facebook/presto/spi/statistics/TestHistogramCalculator.java @@ -0,0 +1,101 @@ +/* + * 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.statistics; + +import com.facebook.presto.common.predicate.Range; +import org.testng.annotations.Test; + +import static com.facebook.presto.common.type.DoubleType.DOUBLE; +import static com.facebook.presto.spi.statistics.HistogramCalculator.calculateFilterFactor; +import static java.lang.Double.NEGATIVE_INFINITY; +import static java.lang.Double.NaN; +import static java.lang.Double.POSITIVE_INFINITY; +import static org.testng.Assert.assertEquals; + +public class TestHistogramCalculator +{ + @Test + public void testCalculateFilterFactor() + { + Range zeroToTen = range(0, 10); + Range empty = Range.range(DOUBLE, NaN, true, NaN, true); + + // Equal ranges + assertFilterFactor(Estimate.of(1.0), zeroToTen, 10, uniformHist(0, 10), 5); + assertFilterFactor(Estimate.of(1.0), zeroToTen, 10, uniformHist(0, 10), 20); + + // Some overlap + assertFilterFactor(Estimate.of(0.5), range(5, 3000), 5, uniformHist(zeroToTen), 10); + + // Single value overlap + assertFilterFactor(Estimate.of(1.0 / 10), range(3, 3), 1, uniformHist(zeroToTen), 10); + assertFilterFactor(Estimate.of(1.0 / 10), range(10, 100), 357, uniformHist(zeroToTen), 10); + + // No overlap + assertFilterFactor(Estimate.zero(), range(20, 30), 10, uniformHist(zeroToTen), 10); + + // Empty ranges + assertFilterFactor(Estimate.zero(), zeroToTen, 10, uniformHist(empty), 0); + assertFilterFactor(Estimate.zero(), empty, 0, uniformHist(zeroToTen), 10); + + // no test for (empty, empty) since any return value is correct + assertFilterFactor(Estimate.zero(), unboundedRange(), 10, uniformHist(empty), 0); + assertFilterFactor(Estimate.zero(), empty, 0, uniformHist(unboundedRange()), 10); + + // Unbounded (infinite), NDV-based + assertFilterFactor(Estimate.of(0.5), unboundedRange(), 10, uniformHist(unboundedRange()), 20); + assertFilterFactor(Estimate.of(1.0), unboundedRange(), 20, uniformHist(unboundedRange()), 10); + + // NEW TESTS (TPC-H Q2) + // unbounded ranges + assertFilterFactor(Estimate.of(.5), unboundedRange(), 0.5, uniformHist(unboundedRange()), NaN); + // unbounded ranges with limited distinct values + assertFilterFactor(Estimate.of(0.2), unboundedRange(), 1.0, + domainConstrained(unboundedRange(), uniformHist(unboundedRange())), 5.0); + } + + private static Range range(double low, double high) + { + return Range.range(DOUBLE, low, true, high, true); + } + + private static Range unboundedRange() + { + return Range.all(DOUBLE); + } + + private static void assertFilterFactor(Estimate expected, Range range, double distinctValues, ConnectorHistogram histogram, double totalDistinctValues) + { + assertEquals( + calculateFilterFactor(range, distinctValues, histogram, Estimate.estimateFromDouble(totalDistinctValues), true), + expected); + } + + private static ConnectorHistogram uniformHist(Range range) + { + return uniformHist(range.getLow().getObjectValue().map(Double.class::cast).orElse(NEGATIVE_INFINITY), + range.getHigh().getObjectValue().map(Double.class::cast).orElse(POSITIVE_INFINITY)); + } + + private static ConnectorHistogram uniformHist(double low, double high) + { + return new UniformDistributionHistogram(low, high); + } + + private static ConnectorHistogram domainConstrained(Range range, ConnectorHistogram source) + { + return DisjointRangeDomainHistogram.addDisjunction(source, range); + } +} diff --git a/presto-main/src/test/java/com/facebook/presto/cost/TestUniformHistogram.java b/presto-spi/src/test/java/com/facebook/presto/spi/statistics/TestUniformHistogram.java similarity index 93% rename from presto-main/src/test/java/com/facebook/presto/cost/TestUniformHistogram.java rename to presto-spi/src/test/java/com/facebook/presto/spi/statistics/TestUniformHistogram.java index 395bc3f6e751..e1d3dc0b6f16 100644 --- a/presto-main/src/test/java/com/facebook/presto/cost/TestUniformHistogram.java +++ b/presto-spi/src/test/java/com/facebook/presto/spi/statistics/TestUniformHistogram.java @@ -12,11 +12,8 @@ * limitations under the License. */ -package com.facebook.presto.cost; +package com.facebook.presto.spi.statistics; -import com.facebook.presto.spi.statistics.ConnectorHistogram; -import com.facebook.presto.spi.statistics.Estimate; -import com.google.common.base.VerifyException; import org.apache.commons.math3.distribution.RealDistribution; import org.apache.commons.math3.distribution.UniformRealDistribution; import org.testng.annotations.Test; @@ -48,7 +45,7 @@ RealDistribution getDistribution() @Test public void testInvalidConstruction() { - assertThrows(VerifyException.class, () -> new UniformDistributionHistogram(2.0, 1.0)); + assertThrows(IllegalArgumentException.class, () -> new UniformDistributionHistogram(2.0, 1.0)); } @Test From 05bc56ceddaf7debed9f5fd7aa20e07093aea969 Mon Sep 17 00:00:00 2001 From: Zac Blanco Date: Mon, 22 Apr 2024 10:07:53 -0400 Subject: [PATCH 05/25] Add histogram statistic documentation --- .../src/main/sphinx/admin/properties.rst | 12 ++++++++ .../src/main/sphinx/optimizer/statistics.rst | 29 ++++++++++--------- .../src/main/sphinx/sql/show-stats.rst | 3 ++ 3 files changed, 30 insertions(+), 14 deletions(-) diff --git a/presto-docs/src/main/sphinx/admin/properties.rst b/presto-docs/src/main/sphinx/admin/properties.rst index de49d8bcaa87..0669e89b216b 100644 --- a/presto-docs/src/main/sphinx/admin/properties.rst +++ b/presto-docs/src/main/sphinx/admin/properties.rst @@ -902,6 +902,18 @@ Enable retry for failed queries who can potentially be helped by HBO. The corresponding session property is :ref:`admin/properties-session:\`\`retry-query-with-history-based-optimization\`\``. +``optimizer.use-histograms`` +^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +* **Type:** ``boolean`` +* **Default Value:** ``false`` + +Enables the optimizer to use histograms when available to perform cost estimate calculations +during query optimization. When set to ``false``, this parameter does not prevent histograms +from being collected by ``ANALYZE``, but prevents them from being used during query +optimization. This behavior can be controlled on a per-query basis using the +``optimizer_use_histograms`` session property. + Planner Properties ------------------ diff --git a/presto-docs/src/main/sphinx/optimizer/statistics.rst b/presto-docs/src/main/sphinx/optimizer/statistics.rst index a0c562abeb4a..eeb763575c82 100644 --- a/presto-docs/src/main/sphinx/optimizer/statistics.rst +++ b/presto-docs/src/main/sphinx/optimizer/statistics.rst @@ -6,8 +6,9 @@ Presto supports statistics based optimizations for queries. For a query to take advantage of these optimizations, Presto must have statistical information for the tables in that query. -Table statistics are provided to the query planner by connectors. Currently, the -only connector that supports statistics is the :doc:`/connector/hive`. +Table statistics are provided to the query planner by connectors. Implementing +support for table statistics is optional. The decision is left to the authors +of the connector. Table Layouts ------------- @@ -30,23 +31,23 @@ Available Statistics The following statistics are available in Presto: - * For a table: +* For a table: - * **row count**: the total number of rows in the table layout + * **row count**: the total number of rows in the table layout - * For each column in a table: +* For each column in a table: - * **data size**: the size of the data that needs to be read - * **nulls fraction**: the fraction of null values - * **distinct value count**: the number of distinct values - * **low value**: the smallest value in the column - * **high value**: the largest value in the column + * **data size**: the size of the data that needs to be read + * **nulls fraction**: the fraction of null values + * **distinct value count**: the number of distinct values + * **low value**: the smallest value in the column + * **high value**: the largest value in the column + * **histogram**: A connector-dependent histogram data structure. The set of statistics available for a particular query depends on the connector being used and can also vary by table or even by table layout. For example, the Hive connector does not currently provide statistics on data size. -Table statistics can be displayed via the Presto SQL interface using the -:doc:`/sql/show-stats` command. For the Hive connector, refer to the -:ref:`Hive connector ` documentation to learn how to update table -statistics. +Table statistics can be can be fetched using the :doc:`/sql/show-stats` query. +For the Hive connector, refer to the :ref:`Hive connector ` +documentation to learn how to update table statistics. diff --git a/presto-docs/src/main/sphinx/sql/show-stats.rst b/presto-docs/src/main/sphinx/sql/show-stats.rst index 51dae9df5dac..79b5ec48ffa6 100644 --- a/presto-docs/src/main/sphinx/sql/show-stats.rst +++ b/presto-docs/src/main/sphinx/sql/show-stats.rst @@ -60,3 +60,6 @@ The following table lists the returned columns and what statistics they represen - The highest value found in this column - ``NULL`` in the table summary row. Available for columns of DATE, integer, floating-point, and fixed-precision data types. + * - ``histogram`` + - The histogram for this column + - A summary of the underlying histogram is displayed in a human-readable format. ``NULL`` in the table summary row. From 8f3cd138292bedcd28a27ac6583198d956e60731 Mon Sep 17 00:00:00 2001 From: Xiao Du Date: Fri, 13 Dec 2024 13:11:49 -0800 Subject: [PATCH 06/25] Add session properties for scale writer query configs --- .../sphinx/presto_cpp/properties-session.rst | 44 ++++++++++++++++++- .../NativeWorkerSessionPropertyProvider.java | 35 ++++++++++++++- .../presto_cpp/main/SessionProperties.cpp | 41 +++++++++++++++++ .../presto_cpp/main/SessionProperties.h | 27 +++++++++++- .../main/tests/SessionPropertiesTest.cpp | 14 +++++- 5 files changed, 156 insertions(+), 5 deletions(-) diff --git a/presto-docs/src/main/sphinx/presto_cpp/properties-session.rst b/presto-docs/src/main/sphinx/presto_cpp/properties-session.rst index 0fe43e1c08c2..b699b922003a 100644 --- a/presto-docs/src/main/sphinx/presto_cpp/properties-session.rst +++ b/presto-docs/src/main/sphinx/presto_cpp/properties-session.rst @@ -376,4 +376,46 @@ The shard id to be traced. If not specified, all shards will be matched. * **Default value:** ``""`` The regular expression to match a task for tracing. It will be deprecated if there is -no issue with native_query_trace_fragment_id and native_query_trace_shard_id. \ No newline at end of file +no issue with native_query_trace_fragment_id and native_query_trace_shard_id. + +``native_scaled_writer_rebalance_max_memory_usage_ratio`` +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +* **Type:** ``double`` +* **Minimum value:** ``0`` +* **Maximum value:** ``1`` +* **Default value:** ``0.7`` + +The max ratio of a query used memory to its max capacity, and the scale +writer exchange stops scaling writer processing if the query's current +memory usage exceeds this ratio. The value is in the range of (0, 1]. + +``native_scaled_writer_max_partitions_per_writer`` +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +* **Type:** ``integer`` +* **Default value:** ``128`` + +The max number of logical table partitions that can be assigned to a +single table writer thread. The logical table partition is used by local +exchange writer for writer scaling, and multiple physical table +partitions can be mapped to the same logical table partition based on the +hash value of calculated partitioned ids. + +``native_scaled_writer_min_partition_processed_bytes_rebalance_threshold`` +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +* **Type:** ``bigint`` +* **Default value:** ``134217728`` + +Minimum amount of data processed by a logical table partition to trigger +writer scaling if it is detected as overloaded by scale writer exchange. + +``native_scaled_writer_min_processed_bytes_rebalance_threshold`` +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +* **Type:** ``bigint`` +* **Default value:** ``268435456`` + +Minimum amount of data processed by all the logical table partitions to +trigger skewed partition rebalancing by scale writer exchange. diff --git a/presto-main/src/main/java/com/facebook/presto/sessionpropertyproviders/NativeWorkerSessionPropertyProvider.java b/presto-main/src/main/java/com/facebook/presto/sessionpropertyproviders/NativeWorkerSessionPropertyProvider.java index d2411b5a3cb7..08d6c939b6d8 100644 --- a/presto-main/src/main/java/com/facebook/presto/sessionpropertyproviders/NativeWorkerSessionPropertyProvider.java +++ b/presto-main/src/main/java/com/facebook/presto/sessionpropertyproviders/NativeWorkerSessionPropertyProvider.java @@ -22,6 +22,7 @@ import java.util.List; import static com.facebook.presto.spi.session.PropertyMetadata.booleanProperty; +import static com.facebook.presto.spi.session.PropertyMetadata.doubleProperty; import static com.facebook.presto.spi.session.PropertyMetadata.integerProperty; import static com.facebook.presto.spi.session.PropertyMetadata.longProperty; import static com.facebook.presto.spi.session.PropertyMetadata.stringProperty; @@ -68,6 +69,10 @@ public class NativeWorkerSessionPropertyProvider public static final String NATIVE_PREFIXSORT_NORMALIZED_KEY_MAX_BYTES = "native_prefixsort_normalized_key_max_bytes"; public static final String NATIVE_PREFIXSORT_MIN_ROWS = "native_prefixsort_min_rows"; public static final String NATIVE_OP_TRACE_DIR_CREATE_CONFIG = "native_op_trace_directory_create_config"; + public static final String NATIVE_SCALED_WRITER_REBALANCE_MAX_MEMORY_USAGE_RATIO = "native_scaled_writer_rebalance_max_memory_usage_ratio"; + public static final String NATIVE_SCALED_WRITER_MAX_PARTITIONS_PER_WRITER = "native_scaled_writer_max_partitions_per_writer"; + public static final String NATIVE_SCALED_WRITER_MIN_PARTITION_PROCESSED_BYTES_REBALANCE_THRESHOLD = "native_scaled_writer_min_partition_processed_bytes_rebalance_threshold"; + public static final String NATIVE_SCALED_WRITER_MIN_PROCESSED_BYTES_REBALANCE_THRESHOLD = "native_scaled_writer_min_processed_bytes_rebalance_threshold"; private final List> sessionProperties; @Inject @@ -133,7 +138,7 @@ public NativeWorkerSessionPropertyProvider(FeaturesConfig featuresConfig) longProperty( NATIVE_WRITER_FLUSH_THRESHOLD_BYTES, "Native Execution only. Minimum memory footprint size required to reclaim memory from a file " + - "writer by flushing its buffered data to disk.", + "writer by flushing its buffered data to disk.", 96L << 20, false), booleanProperty( @@ -276,6 +281,34 @@ public NativeWorkerSessionPropertyProvider(FeaturesConfig featuresConfig) "Minimum number of rows to use prefix-sort. " + "The default value (130) has been derived using micro-benchmarking.", 130, + !nativeExecution), + doubleProperty( + NATIVE_SCALED_WRITER_REBALANCE_MAX_MEMORY_USAGE_RATIO, + "The max ratio of a query used memory to its max capacity, " + + "and the scale writer exchange stops scaling writer processing if the query's current " + + "memory usage exceeds this ratio. The value is in the range of (0, 1].", + 0.7, + !nativeExecution), + integerProperty( + NATIVE_SCALED_WRITER_MAX_PARTITIONS_PER_WRITER, + "The max number of logical table partitions that can be assigned to a " + + "single table writer thread. The logical table partition is used by local " + + "exchange writer for writer scaling, and multiple physical table " + + "partitions can be mapped to the same logical table partition based on the " + + "hash value of calculated partitioned ids", + 128, + !nativeExecution), + longProperty( + NATIVE_SCALED_WRITER_MIN_PARTITION_PROCESSED_BYTES_REBALANCE_THRESHOLD, + "Minimum amount of data processed by a logical table partition " + + "to trigger writer scaling if it is detected as overloaded by scale writer exchange.", + 128L << 20, + !nativeExecution), + longProperty( + NATIVE_SCALED_WRITER_MIN_PROCESSED_BYTES_REBALANCE_THRESHOLD, + "Minimum amount of data processed by all the logical table partitions " + + "to trigger skewed partition rebalancing by scale writer exchange.", + 256L << 20, !nativeExecution)); } diff --git a/presto-native-execution/presto_cpp/main/SessionProperties.cpp b/presto-native-execution/presto_cpp/main/SessionProperties.cpp index 1c50d2d91d95..30ef49e3420b 100644 --- a/presto-native-execution/presto_cpp/main/SessionProperties.cpp +++ b/presto-native-execution/presto_cpp/main/SessionProperties.cpp @@ -401,6 +401,47 @@ SessionProperties::SessionProperties() { false, QueryConfig::kPrefixSortMinRows, std::to_string(c.prefixSortMinRows())); + + addSessionProperty( + kScaleWriterRebalanceMaxMemoryUsageRatio, + "The max ratio of a query used memory to its max capacity, " + "and the scale writer exchange stops scaling writer processing if the query's current " + "memory usage exceeds this ratio. The value is in the range of (0, 1].", + DOUBLE(), + false, + QueryConfig::kScaleWriterRebalanceMaxMemoryUsageRatio, + std::to_string(c.scaleWriterRebalanceMaxMemoryUsageRatio())); + + addSessionProperty( + kScaleWriterMaxPartitionsPerWriter, + "The max number of logical table partitions that can be assigned to a " + "single table writer thread. The logical table partition is used by local " + "exchange writer for writer scaling, and multiple physical table " + "partitions can be mapped to the same logical table partition based on the " + "hash value of calculated partitioned ids.", + INTEGER(), + false, + QueryConfig::kScaleWriterMaxPartitionsPerWriter, + std::to_string(c.scaleWriterMaxPartitionsPerWriter())); + + addSessionProperty( + kScaleWriterMinPartitionProcessedBytesRebalanceThreshold, + "Minimum amount of data processed by a logical table partition " + "to trigger writer scaling if it is detected as overloaded by scale writer exchange.", + BIGINT(), + false, + QueryConfig::kScaleWriterMinPartitionProcessedBytesRebalanceThreshold, + std::to_string( + c.scaleWriterMinPartitionProcessedBytesRebalanceThreshold())); + + addSessionProperty( + kScaleWriterMinProcessedBytesRebalanceThreshold, + "Minimum amount of data processed by all the logical table partitions " + "to trigger skewed partition rebalancing by scale writer exchange.", + BIGINT(), + false, + QueryConfig::kScaleWriterMinProcessedBytesRebalanceThreshold, + std::to_string(c.scaleWriterMinProcessedBytesRebalanceThreshold())); } const std::unordered_map>& diff --git a/presto-native-execution/presto_cpp/main/SessionProperties.h b/presto-native-execution/presto_cpp/main/SessionProperties.h index ab6a36565dd6..3156c66de756 100644 --- a/presto-native-execution/presto_cpp/main/SessionProperties.h +++ b/presto-native-execution/presto_cpp/main/SessionProperties.h @@ -178,6 +178,31 @@ class SessionProperties { static constexpr const char* kSelectiveNimbleReaderEnabled = "native_selective_nimble_reader_enabled"; + /// The max ratio of a query used memory to its max capacity, and the scale + /// writer exchange stops scaling writer processing if the query's current + /// memory usage exceeds this ratio. The value is in the range of (0, 1]. + static constexpr const char* kScaleWriterRebalanceMaxMemoryUsageRatio = + "native_scaled_writer_rebalance_max_memory_usage_ratio"; + + /// The max number of logical table partitions that can be assigned to a + /// single table writer thread. The logical table partition is used by local + /// exchange writer for writer scaling, and multiple physical table + /// partitions can be mapped to the same logical table partition based on the + /// hash value of calculated partitioned ids. + static constexpr const char* kScaleWriterMaxPartitionsPerWriter = + "native_scaled_writer_max_partitions_per_writer"; + + /// Minimum amount of data processed by a logical table partition to trigger + /// writer scaling if it is detected as overloaded by scale writer exchange. + static constexpr const char* + kScaleWriterMinPartitionProcessedBytesRebalanceThreshold = + "native_scaled_writer_min_partition_processed_bytes_rebalance_threshold"; + + /// Minimum amount of data processed by all the logical table partitions to + /// trigger skewed partition rebalancing by scale writer exchange. + static constexpr const char* kScaleWriterMinProcessedBytesRebalanceThreshold = + "native_scaled_writer_min_processed_bytes_rebalance_threshold"; + /// Enable timezone-less timestamp conversions. static constexpr const char* kLegacyTimestamp = "legacy_timestamp"; @@ -246,7 +271,7 @@ class SessionProperties { /// prefix keys, which might have potential risk of running out of server /// memory. static constexpr const char* kSpillPrefixSortEnabled = - "spill_prefixsort_enabled"; + "native_spill_prefixsort_enabled"; /// Maximum number of bytes to use for the normalized key in prefix-sort. Use /// 0 to disable prefix-sort. diff --git a/presto-native-execution/presto_cpp/main/tests/SessionPropertiesTest.cpp b/presto-native-execution/presto_cpp/main/tests/SessionPropertiesTest.cpp index 5da79aac1081..6b6ad6533466 100644 --- a/presto-native-execution/presto_cpp/main/tests/SessionPropertiesTest.cpp +++ b/presto-native-execution/presto_cpp/main/tests/SessionPropertiesTest.cpp @@ -26,11 +26,21 @@ TEST_F(SessionPropertiesTest, validateMapping) { const std::vector names = { SessionProperties::kLegacyTimestamp, SessionProperties::kDriverCpuTimeSliceLimitMs, - SessionProperties::kSpillCompressionCodec}; + SessionProperties::kSpillCompressionCodec, + SessionProperties::kScaleWriterRebalanceMaxMemoryUsageRatio, + SessionProperties::kScaleWriterMaxPartitionsPerWriter, + SessionProperties:: + kScaleWriterMinPartitionProcessedBytesRebalanceThreshold, + SessionProperties::kScaleWriterMinProcessedBytesRebalanceThreshold}; const std::vector veloxConfigNames = { core::QueryConfig::kAdjustTimestampToTimezone, core::QueryConfig::kDriverCpuTimeSliceLimitMs, - core::QueryConfig::kSpillCompressionKind}; + core::QueryConfig::kSpillCompressionKind, + core::QueryConfig::kScaleWriterRebalanceMaxMemoryUsageRatio, + core::QueryConfig::kScaleWriterMaxPartitionsPerWriter, + core::QueryConfig:: + kScaleWriterMinPartitionProcessedBytesRebalanceThreshold, + core::QueryConfig::kScaleWriterMinProcessedBytesRebalanceThreshold}; auto sessionProperties = SessionProperties().getSessionProperties(); const auto len = names.size(); for (auto i = 0; i < len; i++) { From 2183c0a1ef697f18004f0d12f3590028e8310998 Mon Sep 17 00:00:00 2001 From: Ge Gao Date: Fri, 13 Dec 2024 19:25:19 -0800 Subject: [PATCH 07/25] [Native] Make PrestoServer create and pass connectorCpuExecutor to Connector Create a CPUThreadPoolExecutor data member, connectorCpuExecutor_, for PrestoServer. Pass it to every created Connector. Add a new config `connector.num-cpu-threads-hw-multiplier` to control how many threads would be used for the executor. `connector.num-cpu-threads-hw-multiplier` will set connectorCpuExecutor_ to nullptr. Make a process-wise managed CPUThreadPoolExecutor instance available to all connectors. Connector could schedule CPU-bound async operators to it so that they will not occupy the driver thread pool. --- .../presto_cpp/main/PrestoServer.cpp | 26 ++++++++++++++++++- .../presto_cpp/main/PrestoServer.h | 3 +++ .../presto_cpp/main/common/Configs.cpp | 5 ++++ .../presto_cpp/main/common/Configs.h | 10 +++++++ presto-native-execution/velox | 2 +- 5 files changed, 44 insertions(+), 2 deletions(-) diff --git a/presto-native-execution/presto_cpp/main/PrestoServer.cpp b/presto-native-execution/presto_cpp/main/PrestoServer.cpp index a01eb58bdd17..84b3b26fde0a 100644 --- a/presto-native-execution/presto_cpp/main/PrestoServer.cpp +++ b/presto-native-execution/presto_cpp/main/PrestoServer.cpp @@ -665,6 +665,15 @@ void PrestoServer::run() { // HTTP IO executor threads are joined. driverExecutor_.reset(); + if (connectorCpuExecutor_) { + PRESTO_SHUTDOWN_LOG(INFO) + << "Joining Connector CPU Executor '" + << connectorCpuExecutor_->getName() + << "': threads: " << connectorCpuExecutor_->numActiveThreads() << "/" + << connectorCpuExecutor_->numThreads(); + connectorCpuExecutor_->join(); + } + if (connectorIoExecutor_) { PRESTO_SHUTDOWN_LOG(INFO) << "Joining Connector IO Executor '" << connectorIoExecutor_->getName() @@ -1172,6 +1181,20 @@ std::vector PrestoServer::registerConnectors( const fs::path& configDirectoryPath) { static const std::string kPropertiesExtension = ".properties"; + const auto numConnectorCpuThreads = std::max( + SystemConfig::instance()->connectorNumCpuThreadsHwMultiplier() * + std::thread::hardware_concurrency(), + 0); + if (numConnectorCpuThreads > 0) { + connectorCpuExecutor_ = std::make_unique( + numConnectorCpuThreads, + std::make_shared("Connector")); + + PRESTO_STARTUP_LOG(INFO) + << "Connector CPU executor has " << connectorCpuExecutor_->numThreads() + << " threads."; + } + const auto numConnectorIoThreads = std::max( SystemConfig::instance()->connectorNumIoThreadsHwMultiplier() * std::thread::hardware_concurrency(), @@ -1218,7 +1241,8 @@ std::vector PrestoServer::registerConnectors( ->newConnector( catalogName, std::move(properties), - connectorIoExecutor_.get()); + connectorIoExecutor_.get(), + connectorCpuExecutor_.get()); velox::connector::registerConnector(connector); } } diff --git a/presto-native-execution/presto_cpp/main/PrestoServer.h b/presto-native-execution/presto_cpp/main/PrestoServer.h index d364befaa1b6..615d6bfb910d 100644 --- a/presto-native-execution/presto_cpp/main/PrestoServer.h +++ b/presto-native-execution/presto_cpp/main/PrestoServer.h @@ -235,6 +235,9 @@ class PrestoServer { // Executor for background writing into SSD cache. std::unique_ptr cacheExecutor_; + // Executor for async execution for connectors. + std::unique_ptr connectorCpuExecutor_; + // Executor for async IO for connectors. std::unique_ptr connectorIoExecutor_; diff --git a/presto-native-execution/presto_cpp/main/common/Configs.cpp b/presto-native-execution/presto_cpp/main/common/Configs.cpp index 0929f43436a8..a3227719eb9e 100644 --- a/presto-native-execution/presto_cpp/main/common/Configs.cpp +++ b/presto-native-execution/presto_cpp/main/common/Configs.cpp @@ -155,6 +155,7 @@ SystemConfig::SystemConfig() { NONE_PROP(kHttpsClientCertAndKeyPath), NUM_PROP(kExchangeHttpClientNumIoThreadsHwMultiplier, 1.0), NUM_PROP(kExchangeHttpClientNumCpuThreadsHwMultiplier, 1.0), + NUM_PROP(kConnectorNumCpuThreadsHwMultiplier, 0.0), NUM_PROP(kConnectorNumIoThreadsHwMultiplier, 1.0), NUM_PROP(kDriverNumCpuThreadsHwMultiplier, 4.0), BOOL_PROP(kDriverThreadsBatchSchedulingEnabled, false), @@ -375,6 +376,10 @@ double SystemConfig::exchangeHttpClientNumCpuThreadsHwMultiplier() const { .value(); } +double SystemConfig::connectorNumCpuThreadsHwMultiplier() const { + return optionalProperty(kConnectorNumCpuThreadsHwMultiplier).value(); +} + double SystemConfig::connectorNumIoThreadsHwMultiplier() const { return optionalProperty(kConnectorNumIoThreadsHwMultiplier).value(); } diff --git a/presto-native-execution/presto_cpp/main/common/Configs.h b/presto-native-execution/presto_cpp/main/common/Configs.h index 56f54e30f3d4..fcd17794c1c6 100644 --- a/presto-native-execution/presto_cpp/main/common/Configs.h +++ b/presto-native-execution/presto_cpp/main/common/Configs.h @@ -213,6 +213,14 @@ class SystemConfig : public ConfigBase { static constexpr std::string_view kHttpsClientCertAndKeyPath{ "https-client-cert-key-path"}; + /// Floating point number used in calculating how many threads we would use + /// for CPU executor for connectors mainly for async operators: + /// hw_concurrency x multiplier. + /// If 0.0 then connector CPU executor would not be created. + /// 0.0 is default. + static constexpr std::string_view kConnectorNumCpuThreadsHwMultiplier{ + "connector.num-cpu-threads-hw-multiplier"}; + /// Floating point number used in calculating how many threads we would use /// for IO executor for connectors mainly to do preload/prefetch: /// hw_concurrency x multiplier. @@ -724,6 +732,8 @@ class SystemConfig : public ConfigBase { double exchangeHttpClientNumCpuThreadsHwMultiplier() const; + double connectorNumCpuThreadsHwMultiplier() const; + double connectorNumIoThreadsHwMultiplier() const; double driverNumCpuThreadsHwMultiplier() const; diff --git a/presto-native-execution/velox b/presto-native-execution/velox index 960c2af9be8a..12942c1eb76d 160000 --- a/presto-native-execution/velox +++ b/presto-native-execution/velox @@ -1 +1 @@ -Subproject commit 960c2af9be8a5992dd4528f945fb63ca6dae2f92 +Subproject commit 12942c1eb76de019b775f4b207bc4595d8ace5c0 From 33d7c035be62a4aca32e8eb5ef0ecb16f8234a6c Mon Sep 17 00:00:00 2001 From: Abe Varghese <148206484+abevk2023@users.noreply.github.com> Date: Tue, 17 Dec 2024 02:36:56 +0530 Subject: [PATCH 08/25] Fixed Functional Test Failures (#24123) Fix functional test failures Fixed mount issues and docker host for running in pipeline --- presto-native-execution/pom.xml | 6 +++ .../nativeworker/ContainerQueryRunner.java | 39 ++++++++++--------- .../ContainerQueryRunnerUtils.java | 25 ++++++------ 3 files changed, 40 insertions(+), 30 deletions(-) diff --git a/presto-native-execution/pom.xml b/presto-native-execution/pom.xml index 3dcfe4a02a1d..79271d2bb3ca 100644 --- a/presto-native-execution/pom.xml +++ b/presto-native-execution/pom.xml @@ -244,6 +244,12 @@ presto-jdbc test + + org.apache.commons + commons-lang3 + 3.14.0 + test + diff --git a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/ContainerQueryRunner.java b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/ContainerQueryRunner.java index d32eea55c392..9cd88e2ade8d 100644 --- a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/ContainerQueryRunner.java +++ b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/ContainerQueryRunner.java @@ -30,14 +30,14 @@ import com.facebook.presto.testing.QueryRunner; import com.facebook.presto.testing.TestingAccessControlManager; import com.facebook.presto.transaction.TransactionManager; -import org.testcontainers.containers.BindMode; import org.testcontainers.containers.GenericContainer; import org.testcontainers.containers.Network; import org.testcontainers.containers.wait.strategy.Wait; +import org.testcontainers.utility.MountableFile; import java.io.IOException; import java.sql.Connection; -import java.sql.DriverManager; +import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; import java.time.Duration; @@ -50,6 +50,7 @@ import java.util.logging.Logger; import static com.facebook.presto.testing.TestingSession.testSessionBuilder; +import static java.sql.DriverManager.getConnection; public class ContainerQueryRunner implements QueryRunner @@ -72,7 +73,6 @@ public class ContainerQueryRunner private final String schema; private final int numberOfWorkers; private Connection connection; - private Statement statement; public ContainerQueryRunner() throws InterruptedException, IOException @@ -97,19 +97,20 @@ public ContainerQueryRunner(int coordinatorPort, String catalog, String schema, coordinator.start(); workers.forEach(GenericContainer::start); - logger.info("Presto UI is accessible at http://localhost:" + coordinator.getMappedPort(coordinatorPort)); - TimeUnit.SECONDS.sleep(5); - String url = String.format("jdbc:presto://localhost:%s/%s/%s?%s", + String dockerHostIp = coordinator.getHost(); + logger.info("Presto UI is accessible at http://" + dockerHostIp + ":" + coordinator.getMappedPort(coordinatorPort)); + + String url = String.format("jdbc:presto://%s:%s/%s/%s?%s", + dockerHostIp, coordinator.getMappedPort(coordinatorPort), catalog, schema, "timeZoneId=UTC"); try { - Connection connection = DriverManager.getConnection(url, "test", null); - statement = connection.createStatement(); + connection = getConnection(url, "test", null); } catch (SQLException e) { throw new RuntimeException(e); @@ -134,9 +135,10 @@ private GenericContainer createCoordinator() return new GenericContainer<>(PRESTO_COORDINATOR_IMAGE) .withExposedPorts(coordinatorPort) - .withNetwork(network).withNetworkAliases("presto-coordinator") - .withFileSystemBind(BASE_DIR + "/testcontainers/coordinator/etc", "/opt/presto-server/etc", BindMode.READ_WRITE) - .withFileSystemBind(BASE_DIR + "/testcontainers/coordinator/entrypoint.sh", "/opt/entrypoint.sh", BindMode.READ_ONLY) + .withNetwork(network) + .withNetworkAliases("presto-coordinator") + .withCopyFileToContainer(MountableFile.forHostPath(BASE_DIR + "/testcontainers/coordinator/etc"), "/opt/presto-server/etc") + .withCopyFileToContainer(MountableFile.forHostPath(BASE_DIR + "/testcontainers/coordinator/entrypoint.sh"), "/opt/entrypoint.sh") .waitingFor(Wait.forLogMessage(".*======== SERVER STARTED ========.*", 1)) .withStartupTimeout(Duration.ofSeconds(Long.parseLong(CONTAINER_TIMEOUT))); } @@ -151,9 +153,10 @@ private GenericContainer createNativeWorker(int port, String nodeId) ContainerQueryRunnerUtils.createNativeWorkerVeloxProperties(nodeId); return new GenericContainer<>(PRESTO_WORKER_IMAGE) .withExposedPorts(port) - .withNetwork(network).withNetworkAliases(nodeId) - .withFileSystemBind(BASE_DIR + "/testcontainers/" + nodeId + "/etc", "/opt/presto-server/etc", BindMode.READ_ONLY) - .withFileSystemBind(BASE_DIR + "/testcontainers/" + nodeId + "/entrypoint.sh", "/opt/entrypoint.sh", BindMode.READ_ONLY) + .withNetwork(network) + .withNetworkAliases(nodeId) + .withCopyFileToContainer(MountableFile.forHostPath(BASE_DIR + "/testcontainers/" + nodeId + "/etc"), "/opt/presto-server/etc") + .withCopyFileToContainer(MountableFile.forHostPath(BASE_DIR + "/testcontainers/" + nodeId + "/entrypoint.sh"), "/opt/entrypoint.sh") .waitingFor(Wait.forLogMessage(".*Announcement succeeded: HTTP 202.*", 1)); } @@ -297,12 +300,12 @@ public Session getDefaultSession() public MaterializedResult execute(Session session, String sql) { try { - return ContainerQueryRunnerUtils - .toMaterializedResult( - statement.executeQuery(sql)); + Statement statement = connection.createStatement(); + ResultSet resultSet = statement.executeQuery(sql); + return ContainerQueryRunnerUtils.toMaterializedResult(resultSet); } catch (SQLException e) { - throw new RuntimeException(e); + throw new RuntimeException("Error executing query: " + sql, e); } } } diff --git a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/ContainerQueryRunnerUtils.java b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/ContainerQueryRunnerUtils.java index 6f29f57b3da9..3a6a98da1ab8 100644 --- a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/ContainerQueryRunnerUtils.java +++ b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/ContainerQueryRunnerUtils.java @@ -28,6 +28,8 @@ import com.facebook.presto.common.type.TimestampWithTimeZoneType; import com.facebook.presto.common.type.TinyintType; import com.facebook.presto.common.type.Type; +import com.facebook.presto.common.type.TypeSignature; +import com.facebook.presto.common.type.TypeSignatureParameter; import com.facebook.presto.common.type.UnknownType; import com.facebook.presto.common.type.VarbinaryType; import com.facebook.presto.common.type.VarcharType; @@ -204,10 +206,6 @@ public static void createPropertiesFile(String filePath, Properties properties) parentDir.mkdirs(); } - if (file.exists()) { - throw new IOException("File exists: " + filePath); - } - try (OutputStreamWriter writer = new OutputStreamWriter(new FileOutputStream(file), StandardCharsets.UTF_8)) { for (String key : properties.stringPropertyNames()) { writer.write(key + "=" + properties.getProperty(key) + "\n"); @@ -224,10 +222,6 @@ public static void createScriptFile(String filePath, String scriptContent) parentDir.mkdirs(); } - if (file.exists()) { - throw new IOException("File exists: " + filePath); - } - try (OutputStream output = new FileOutputStream(file); OutputStreamWriter writer = new OutputStreamWriter(output, StandardCharsets.UTF_8)) { writer.write(scriptContent); @@ -251,10 +245,17 @@ public static MaterializedResult toMaterializedResult(ResultSet resultSet) String typeName = metaData.getColumnTypeName(i); if (sqlType == java.sql.Types.ARRAY) { - // Get the base type of the array elements - String arrayElementTypeName = metaData.getColumnTypeName(i); - Type elementType = mapSqlTypeNameToType(arrayElementTypeName); - types.add(new ArrayType(elementType)); + TypeSignature typeSignature = TypeSignature.parseTypeSignature(typeName); + List parameters = typeSignature.getParameters(); + + if (parameters.size() == 1) { + TypeSignature baseTypeSignature = parameters.get(0).getTypeSignature(); + Type elementType = mapSqlTypeNameToType(baseTypeSignature.toString()); + types.add(new ArrayType(elementType)); + } + else { + throw new UnsupportedOperationException("Unsupported ARRAY type with multiple parameters: " + typeName); + } } else if (sqlType == java.sql.Types.STRUCT) { // Handle STRUCT types if necessary From f1e1fe58a8c6a930983befb4d44b1ff6309bf539 Mon Sep 17 00:00:00 2001 From: Feilong Liu Date: Wed, 20 Nov 2024 15:40:27 -0800 Subject: [PATCH 09/25] Add an option to not enforce input partition for join build --- .../facebook/presto/SystemSessionProperties.java | 11 +++++++++++ .../presto/sql/analyzer/FeaturesConfig.java | 14 ++++++++++++++ .../planner/optimizations/AddLocalExchanges.java | 8 +++++++- .../sql/planner/sanity/ValidateStreamingJoins.java | 8 +++++++- .../presto/sql/analyzer/TestFeaturesConfig.java | 3 +++ 5 files changed, 42 insertions(+), 2 deletions(-) diff --git a/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java b/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java index 8ab8d397d521..8de9af7667eb 100644 --- a/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java +++ b/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java @@ -334,6 +334,7 @@ public final class SystemSessionProperties private static final String NATIVE_EXECUTION_PROGRAM_ARGUMENTS = "native_execution_program_arguments"; public static final String NATIVE_EXECUTION_PROCESS_REUSE_ENABLED = "native_execution_process_reuse_enabled"; public static final String NATIVE_MIN_COLUMNAR_ENCODING_CHANNELS_TO_PREFER_ROW_WISE_ENCODING = "native_min_columnar_encoding_channels_to_prefer_row_wise_encoding"; + public static final String NATIVE_ENFORCE_JOIN_BUILD_INPUT_PARTITION = "native_enforce_join_build_input_partition"; private final List> sessionProperties; @@ -1543,6 +1544,11 @@ public SystemSessionProperties( "Enable reuse the native process within the same JVM", true, false), + booleanProperty( + NATIVE_ENFORCE_JOIN_BUILD_INPUT_PARTITION, + "Enforce that the join build input is partitioned on join key", + featuresConfig.isNativeEnforceJoinBuildInputPartition(), + false), booleanProperty( RANDOMIZE_OUTER_JOIN_NULL_KEY, "(Deprecated) Randomize null join key for outer join", @@ -2888,6 +2894,11 @@ public static boolean isNativeExecutionProcessReuseEnabled(Session session) return session.getSystemProperty(NATIVE_EXECUTION_PROCESS_REUSE_ENABLED, Boolean.class); } + public static boolean isNativeJoinBuildPartitionEnforced(Session session) + { + return session.getSystemProperty(NATIVE_ENFORCE_JOIN_BUILD_INPUT_PARTITION, Boolean.class); + } + public static RandomizeOuterJoinNullKeyStrategy getRandomizeOuterJoinNullKeyStrategy(Session session) { // If RANDOMIZE_OUTER_JOIN_NULL_KEY is set to true, return always enabled, otherwise get strategy from RANDOMIZE_OUTER_JOIN_NULL_KEY_STRATEGY diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java index f7ea9174fe8e..55a812e7054d 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java @@ -238,6 +238,7 @@ public class FeaturesConfig private String nativeExecutionExecutablePath = "./presto_server"; private String nativeExecutionProgramArguments = ""; private boolean nativeExecutionProcessReuseEnabled = true; + private boolean nativeEnforceJoinBuildInputPartition = true; private boolean randomizeOuterJoinNullKey; private RandomizeOuterJoinNullKeyStrategy randomizeOuterJoinNullKeyStrategy = RandomizeOuterJoinNullKeyStrategy.DISABLED; private ShardedJoinStrategy shardedJoinStrategy = ShardedJoinStrategy.DISABLED; @@ -2318,6 +2319,19 @@ public boolean isNativeExecutionProcessReuseEnabled() return this.nativeExecutionProcessReuseEnabled; } + @Config("native-enforce-join-build-input-partition") + @ConfigDescription("Enforce that the join build input is partitioned on join key") + public FeaturesConfig setNativeEnforceJoinBuildInputPartition(boolean nativeEnforceJoinBuildInputPartition) + { + this.nativeEnforceJoinBuildInputPartition = nativeEnforceJoinBuildInputPartition; + return this; + } + + public boolean isNativeEnforceJoinBuildInputPartition() + { + return this.nativeEnforceJoinBuildInputPartition; + } + public boolean isRandomizeOuterJoinNullKeyEnabled() { return randomizeOuterJoinNullKey; diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/AddLocalExchanges.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/AddLocalExchanges.java index 432b031f9aa6..ebec1d015029 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/AddLocalExchanges.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/AddLocalExchanges.java @@ -72,6 +72,7 @@ import static com.facebook.presto.SystemSessionProperties.isDistributedSortEnabled; import static com.facebook.presto.SystemSessionProperties.isEnforceFixedDistributionForOutputOperator; import static com.facebook.presto.SystemSessionProperties.isJoinSpillingEnabled; +import static com.facebook.presto.SystemSessionProperties.isNativeJoinBuildPartitionEnforced; import static com.facebook.presto.SystemSessionProperties.isQuickDistinctLimitEnabled; import static com.facebook.presto.SystemSessionProperties.isSegmentedAggregationEnabled; import static com.facebook.presto.SystemSessionProperties.isSpillEnabled; @@ -831,7 +832,12 @@ public PlanWithProperties visitJoin(JoinNode node, StreamPreferredProperties par .collect(toImmutableList()); StreamPreferredProperties buildPreference; if (getTaskConcurrency(session) > 1) { - buildPreference = exactlyPartitionedOn(buildHashVariables); + if (nativeExecution && !isNativeJoinBuildPartitionEnforced(session)) { + buildPreference = defaultParallelism(session); + } + else { + buildPreference = exactlyPartitionedOn(buildHashVariables); + } } else { buildPreference = singleStream(); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/ValidateStreamingJoins.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/ValidateStreamingJoins.java index 0db581314f7b..52196b42ad56 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/ValidateStreamingJoins.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/ValidateStreamingJoins.java @@ -31,6 +31,7 @@ import static com.facebook.presto.SystemSessionProperties.getTaskConcurrency; import static com.facebook.presto.SystemSessionProperties.isJoinSpillingEnabled; +import static com.facebook.presto.SystemSessionProperties.isNativeJoinBuildPartitionEnforced; import static com.facebook.presto.SystemSessionProperties.isSpillEnabled; import static com.facebook.presto.sql.planner.optimizations.PlanNodeSearcher.searchFrom; import static com.facebook.presto.sql.planner.optimizations.StreamPreferredProperties.defaultParallelism; @@ -89,7 +90,12 @@ public Void visitJoin(JoinNode node, Void context) .collect(toImmutableList()); StreamPreferredProperties requiredBuildProperty; if (getTaskConcurrency(session) > 1) { - requiredBuildProperty = exactlyPartitionedOn(buildJoinVariables); + if (nativeExecutionEnabled && !isNativeJoinBuildPartitionEnforced(session)) { + requiredBuildProperty = defaultParallelism(session); + } + else { + requiredBuildProperty = exactlyPartitionedOn(buildJoinVariables); + } } else { requiredBuildProperty = singleStream(); diff --git a/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java b/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java index ff8154866a60..0fd3a9f66209 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java @@ -203,6 +203,7 @@ public void testDefaults() .setNativeExecutionExecutablePath("./presto_server") .setNativeExecutionProgramArguments("") .setNativeExecutionProcessReuseEnabled(true) + .setNativeEnforceJoinBuildInputPartition(true) .setRandomizeOuterJoinNullKeyEnabled(false) .setRandomizeOuterJoinNullKeyStrategy(RandomizeOuterJoinNullKeyStrategy.DISABLED) .setShardedJoinStrategy(FeaturesConfig.ShardedJoinStrategy.DISABLED) @@ -400,6 +401,7 @@ public void testExplicitPropertyMappings() .put("native-execution-executable-path", "/bin/echo") .put("native-execution-program-arguments", "--v 1") .put("native-execution-process-reuse-enabled", "false") + .put("native-enforce-join-build-input-partition", "false") .put("optimizer.randomize-outer-join-null-key", "true") .put("optimizer.randomize-outer-join-null-key-strategy", "key_from_outer_join") .put("optimizer.sharded-join-strategy", "cost_based") @@ -595,6 +597,7 @@ public void testExplicitPropertyMappings() .setNativeExecutionExecutablePath("/bin/echo") .setNativeExecutionProgramArguments("--v 1") .setNativeExecutionProcessReuseEnabled(false) + .setNativeEnforceJoinBuildInputPartition(false) .setRandomizeOuterJoinNullKeyEnabled(true) .setRandomizeOuterJoinNullKeyStrategy(RandomizeOuterJoinNullKeyStrategy.KEY_FROM_OUTER_JOIN) .setShardedJoinStrategy(FeaturesConfig.ShardedJoinStrategy.COST_BASED) From 37c6bf9ecb60d673677c72eb05789d8fea83e367 Mon Sep 17 00:00:00 2001 From: Deepa-George Date: Sun, 29 Sep 2024 15:23:12 +0530 Subject: [PATCH 10/25] Add support for ALTER VIEW [ IF EXISTS ] RENAME TO statement --- .../sql/analyzer/SemanticErrorCode.java | 2 + .../sql/analyzer/utils/StatementUtils.java | 2 + presto-docs/src/main/sphinx/sql.rst | 1 + .../src/main/sphinx/sql/alter-view.rst | 42 +++++++ .../presto/hive/metastore/MetastoreUtil.java | 5 + .../metastore/file/FileHiveMetastore.java | 4 +- .../hive/security/LegacyAccessControl.java | 5 + .../security/SqlStandardAccessControl.java | 19 +++ .../SystemTableAwareAccessControl.java | 6 + .../presto/execution/RenameViewTask.java | 79 +++++++++++++ .../metadata/DelegatingMetadataManager.java | 6 + .../facebook/presto/metadata/Metadata.java | 5 + .../presto/metadata/MetadataManager.java | 10 ++ .../presto/security/AccessControlManager.java | 17 +++ .../security/AllowAllSystemAccessControl.java | 5 + .../FileBasedSystemAccessControl.java | 9 ++ .../sql/analyzer/StatementAnalyzer.java | 7 ++ .../presto/testing/LocalQueryRunner.java | 3 + .../testing/TestingAccessControlManager.java | 15 ++- .../util/PrestoDataDefBindingHelper.java | 3 + .../presto/metadata/AbstractMockMetadata.java | 6 + .../security/TestAccessControlManager.java | 6 + .../com/facebook/presto/sql/parser/SqlBase.g4 | 2 + .../com/facebook/presto/sql/SqlFormatter.java | 15 +++ .../presto/sql/parser/AstBuilder.java | 7 ++ .../facebook/presto/sql/tree/AstVisitor.java | 5 + .../facebook/presto/sql/tree/RenameView.java | 108 ++++++++++++++++++ .../presto/sql/parser/TestSqlParser.java | 8 ++ .../base/security/AllowAllAccessControl.java | 5 + .../base/security/FileBasedAccessControl.java | 9 ++ .../ForwardingConnectorAccessControl.java | 6 + .../ForwardingSystemAccessControl.java | 6 + .../base/security/ReadOnlyAccessControl.java | 7 ++ .../security/TestFileBasedAccessControl.java | 7 ++ .../src/test/resources/table.json | 5 + .../spi/connector/ConnectorAccessControl.java | 11 ++ .../spi/connector/ConnectorMetadata.java | 8 ++ .../ClassLoaderSafeConnectorMetadata.java | 8 ++ .../presto/spi/security/AccessControl.java | 7 ++ .../spi/security/AccessDeniedException.java | 10 ++ .../spi/security/AllowAllAccessControl.java | 5 + .../spi/security/DenyAllAccessControl.java | 7 ++ .../spi/security/SystemAccessControl.java | 11 ++ 43 files changed, 510 insertions(+), 4 deletions(-) create mode 100644 presto-docs/src/main/sphinx/sql/alter-view.rst create mode 100644 presto-main/src/main/java/com/facebook/presto/execution/RenameViewTask.java create mode 100644 presto-parser/src/main/java/com/facebook/presto/sql/tree/RenameView.java diff --git a/presto-analyzer/src/main/java/com/facebook/presto/sql/analyzer/SemanticErrorCode.java b/presto-analyzer/src/main/java/com/facebook/presto/sql/analyzer/SemanticErrorCode.java index 66b6d7330259..d5492e6bb693 100644 --- a/presto-analyzer/src/main/java/com/facebook/presto/sql/analyzer/SemanticErrorCode.java +++ b/presto-analyzer/src/main/java/com/facebook/presto/sql/analyzer/SemanticErrorCode.java @@ -83,6 +83,8 @@ public enum SemanticErrorCode VIEW_IS_STALE, VIEW_IS_RECURSIVE, MATERIALIZED_VIEW_IS_RECURSIVE, + MISSING_VIEW, + VIEW_ALREADY_EXISTS, NON_NUMERIC_SAMPLE_PERCENTAGE, diff --git a/presto-analyzer/src/main/java/com/facebook/presto/sql/analyzer/utils/StatementUtils.java b/presto-analyzer/src/main/java/com/facebook/presto/sql/analyzer/utils/StatementUtils.java index 0a2b34bebef1..bb16839f8728 100644 --- a/presto-analyzer/src/main/java/com/facebook/presto/sql/analyzer/utils/StatementUtils.java +++ b/presto-analyzer/src/main/java/com/facebook/presto/sql/analyzer/utils/StatementUtils.java @@ -51,6 +51,7 @@ import com.facebook.presto.sql.tree.RenameColumn; import com.facebook.presto.sql.tree.RenameSchema; import com.facebook.presto.sql.tree.RenameTable; +import com.facebook.presto.sql.tree.RenameView; import com.facebook.presto.sql.tree.ResetSession; import com.facebook.presto.sql.tree.Revoke; import com.facebook.presto.sql.tree.RevokeRoles; @@ -134,6 +135,7 @@ private StatementUtils() {} builder.put(AddConstraint.class, QueryType.DATA_DEFINITION); builder.put(AlterColumnNotNull.class, QueryType.DATA_DEFINITION); builder.put(CreateView.class, QueryType.DATA_DEFINITION); + builder.put(RenameView.class, QueryType.DATA_DEFINITION); builder.put(TruncateTable.class, QueryType.DATA_DEFINITION); builder.put(DropView.class, QueryType.DATA_DEFINITION); builder.put(CreateMaterializedView.class, QueryType.DATA_DEFINITION); diff --git a/presto-docs/src/main/sphinx/sql.rst b/presto-docs/src/main/sphinx/sql.rst index 88a4613a2751..84b7ced26d1b 100644 --- a/presto-docs/src/main/sphinx/sql.rst +++ b/presto-docs/src/main/sphinx/sql.rst @@ -10,6 +10,7 @@ This chapter describes the SQL syntax used in Presto. sql/alter-function sql/alter-schema sql/alter-table + sql/alter-view sql/analyze sql/call sql/commit diff --git a/presto-docs/src/main/sphinx/sql/alter-view.rst b/presto-docs/src/main/sphinx/sql/alter-view.rst new file mode 100644 index 000000000000..dfe57193bfbb --- /dev/null +++ b/presto-docs/src/main/sphinx/sql/alter-view.rst @@ -0,0 +1,42 @@ +========== +ALTER VIEW +========== + +Synopsis +-------- + +.. code-block:: sql + + ALTER VIEW [IF EXISTS] old_view_name RENAME TO new_view_name; + +Description +----------- + +The ``ALTER VIEW [IF EXISTS] RENAME TO`` statement renames an existing view to a +new name. This allows you to change the name of a view without having to drop +and recreate it. The view's definition, security settings, and dependencies +remain unchanged; only the name of the view is updated. + +The optional ``IF EXISTS`` clause prevents an error from being raised if the +view does not exist. Instead, no action is taken, and a notice is issued. + +Renaming a view does not affect the data or structure of the underlying +query used to define the view. Any permissions or dependencies on the +view are retained, and queries or applications using the old name must +be updated to use the new name. + +Examples +-------- + +Rename the view ``users`` to ``people``:: + + ALTER VIEW users RENAME TO people; + +Rename the view ``users`` to ``people`` if view ``users`` exists:: + + ALTER VIEW IF EXISTS users RENAME TO people; + +See Also +-------- + +:doc:`create-view` diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/MetastoreUtil.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/MetastoreUtil.java index 28cb2b84017d..87cd7e7e061c 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/MetastoreUtil.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/MetastoreUtil.java @@ -1026,6 +1026,11 @@ public static boolean isIcebergTable(Map tableParameters) return ICEBERG_TABLE_TYPE_VALUE.equalsIgnoreCase(tableParameters.get(ICEBERG_TABLE_TYPE_NAME)); } + public static boolean isIcebergView(Table table) + { + return "true".equalsIgnoreCase(table.getParameters().get(PRESTO_VIEW_FLAG)); + } + public static PrincipalPrivileges buildInitialPrivilegeSet(String tableOwner) { PrestoPrincipal owner = new PrestoPrincipal(USER, tableOwner); diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/file/FileHiveMetastore.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/file/FileHiveMetastore.java index 5ed409e35366..5556948a9b35 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/file/FileHiveMetastore.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/file/FileHiveMetastore.java @@ -511,13 +511,11 @@ public synchronized MetastoreOperationResult renameTable(MetastoreContext metast requireNonNull(tableName, "tableName is null"); requireNonNull(newDatabaseName, "newDatabaseName is null"); requireNonNull(newTableName, "newTableName is null"); - Table table = getRequiredTable(metastoreContext, databaseName, tableName); getRequiredDatabase(metastoreContext, newDatabaseName); - if (isIcebergTable(table)) { + if (isIcebergTable(table) && !isIcebergView(table)) { throw new PrestoException(NOT_SUPPORTED, "Rename not supported for Iceberg tables"); } - // verify new table does not exist verifyTableNotExists(metastoreContext, newDatabaseName, newTableName); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/security/LegacyAccessControl.java b/presto-hive/src/main/java/com/facebook/presto/hive/security/LegacyAccessControl.java index 2277d1dab0f5..d1b77eb6d685 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/security/LegacyAccessControl.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/security/LegacyAccessControl.java @@ -201,6 +201,11 @@ public void checkCanCreateView(ConnectorTransactionHandle transaction, Connector { } + @Override + public void checkCanRenameView(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName viewName, SchemaTableName newViewName) + { + } + @Override public void checkCanDropView(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName viewName) { diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/security/SqlStandardAccessControl.java b/presto-hive/src/main/java/com/facebook/presto/hive/security/SqlStandardAccessControl.java index 6fd805e34c5e..f55247fb6939 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/security/SqlStandardAccessControl.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/security/SqlStandardAccessControl.java @@ -70,6 +70,7 @@ import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameColumn; import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameSchema; import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameTable; +import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameView; import static com.facebook.presto.spi.security.AccessDeniedException.denyRevokeRoles; import static com.facebook.presto.spi.security.AccessDeniedException.denyRevokeTablePrivilege; import static com.facebook.presto.spi.security.AccessDeniedException.denySelectTable; @@ -451,6 +452,24 @@ public void checkCanCreateView(ConnectorTransactionHandle transaction, Connector } } + @Override + public void checkCanRenameView(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName viewName, SchemaTableName newViewName) + { + MetastoreContext metastoreContext = new MetastoreContext( + identity, context.getQueryId().getId(), + context.getClientInfo(), + context.getClientTags(), + context.getSource(), + Optional.empty(), + false, + HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, + context.getWarningCollector(), + context.getRuntimeStats()); + if (!isTableOwner(transaction, identity, metastoreContext, viewName)) { + denyRenameView(viewName.toString(), newViewName.toString()); + } + } + @Override public void checkCanDropView(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName viewName) { diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/security/SystemTableAwareAccessControl.java b/presto-hive/src/main/java/com/facebook/presto/hive/security/SystemTableAwareAccessControl.java index 9f2b182d39d4..74a433e5d635 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/security/SystemTableAwareAccessControl.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/security/SystemTableAwareAccessControl.java @@ -173,6 +173,12 @@ public void checkCanCreateView(ConnectorTransactionHandle transactionHandle, Con delegate.checkCanCreateView(transactionHandle, identity, context, viewName); } + @Override + public void checkCanRenameView(ConnectorTransactionHandle transactionHandle, ConnectorIdentity identity, AccessControlContext context, SchemaTableName viewName, SchemaTableName newViewName) + { + delegate.checkCanRenameView(transactionHandle, identity, context, viewName, newViewName); + } + @Override public void checkCanDropView(ConnectorTransactionHandle transactionHandle, ConnectorIdentity identity, AccessControlContext context, SchemaTableName viewName) { diff --git a/presto-main/src/main/java/com/facebook/presto/execution/RenameViewTask.java b/presto-main/src/main/java/com/facebook/presto/execution/RenameViewTask.java new file mode 100644 index 000000000000..61c3cadc8619 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/execution/RenameViewTask.java @@ -0,0 +1,79 @@ +/* + * 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.execution; + +import com.facebook.presto.Session; +import com.facebook.presto.common.QualifiedObjectName; +import com.facebook.presto.metadata.Metadata; +import com.facebook.presto.spi.WarningCollector; +import com.facebook.presto.spi.analyzer.ViewDefinition; +import com.facebook.presto.spi.security.AccessControl; +import com.facebook.presto.sql.analyzer.SemanticException; +import com.facebook.presto.sql.tree.Expression; +import com.facebook.presto.sql.tree.RenameView; +import com.facebook.presto.transaction.TransactionManager; +import com.google.common.util.concurrent.ListenableFuture; + +import java.util.List; +import java.util.Optional; + +import static com.facebook.presto.metadata.MetadataUtil.createQualifiedObjectName; +import static com.facebook.presto.sql.analyzer.SemanticErrorCode.MISSING_CATALOG; +import static com.facebook.presto.sql.analyzer.SemanticErrorCode.MISSING_VIEW; +import static com.facebook.presto.sql.analyzer.SemanticErrorCode.NOT_SUPPORTED; +import static com.facebook.presto.sql.analyzer.SemanticErrorCode.VIEW_ALREADY_EXISTS; +import static com.google.common.util.concurrent.Futures.immediateFuture; + +public class RenameViewTask + implements DDLDefinitionTask +{ + @Override + public String getName() + { + return "RENAME VIEW"; + } + + public ListenableFuture execute(RenameView statement, TransactionManager transactionManager, Metadata metadata, AccessControl accessControl, Session session, List parameters, WarningCollector warningCollector) + { + QualifiedObjectName viewName = createQualifiedObjectName(session, statement, statement.getSource()); + + Optional view = metadata.getMetadataResolver(session).getView(viewName); + if (!view.isPresent()) { + if (!statement.isExists()) { + throw new SemanticException(MISSING_VIEW, statement, "View '%s' does not exist", viewName); + } + return immediateFuture(null); + } + + QualifiedObjectName target = createQualifiedObjectName(session, statement, statement.getTarget()); + if (!metadata.getCatalogHandle(session, target.getCatalogName()).isPresent()) { + throw new SemanticException(MISSING_CATALOG, statement, "Target catalog '%s' does not exist", target.getCatalogName()); + } + if (metadata.getMetadataResolver(session).getView(target).isPresent()) { + throw new SemanticException(VIEW_ALREADY_EXISTS, statement, "Target view '%s' already exists", target); + } + if (!viewName.getSchemaName().equals(target.getSchemaName())) { + throw new SemanticException(NOT_SUPPORTED, statement, "View rename across schemas is not supported"); + } + if (!viewName.getCatalogName().equals(target.getCatalogName())) { + throw new SemanticException(NOT_SUPPORTED, statement, "View rename across catalogs is not supported"); + } + + accessControl.checkCanRenameView(session.getRequiredTransactionId(), session.getIdentity(), session.getAccessControlContext(), viewName, target); + + metadata.renameView(session, viewName, target); + + return immediateFuture(null); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/DelegatingMetadataManager.java b/presto-main/src/main/java/com/facebook/presto/metadata/DelegatingMetadataManager.java index 8954086f99db..83cd4594032f 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/DelegatingMetadataManager.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/DelegatingMetadataManager.java @@ -432,6 +432,12 @@ public void createView(Session session, String catalogName, ConnectorTableMetada delegate.createView(session, catalogName, viewMetadata, viewData, replace); } + @Override + public void renameView(Session session, QualifiedObjectName existingViewName, QualifiedObjectName newViewName) + { + delegate.renameView(session, existingViewName, newViewName); + } + @Override public void dropView(Session session, QualifiedObjectName viewName) { diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/Metadata.java b/presto-main/src/main/java/com/facebook/presto/metadata/Metadata.java index 1cc99fcce3b9..4f3f698be42c 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/Metadata.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/Metadata.java @@ -368,6 +368,11 @@ public interface Metadata */ void createView(Session session, String catalogName, ConnectorTableMetadata viewMetadata, String viewData, boolean replace); + /** + * Rename the specified view. + */ + void renameView(Session session, QualifiedObjectName existingViewName, QualifiedObjectName newViewName); + /** * Drops the specified view. */ diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/MetadataManager.java b/presto-main/src/main/java/com/facebook/presto/metadata/MetadataManager.java index 7f9812c48f89..3041d3990fdf 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/MetadataManager.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/MetadataManager.java @@ -1004,6 +1004,16 @@ public void createView(Session session, String catalogName, ConnectorTableMetada metadata.createView(session.toConnectorSession(connectorId), viewMetadata, viewData, replace); } + @Override + public void renameView(Session session, QualifiedObjectName source, QualifiedObjectName target) + { + CatalogMetadata catalogMetadata = getCatalogMetadataForWrite(session, target.getCatalogName()); + ConnectorId connectorId = catalogMetadata.getConnectorId(); + ConnectorMetadata metadata = catalogMetadata.getMetadata(); + + metadata.renameView(session.toConnectorSession(connectorId), toSchemaTableName(source), toSchemaTableName(target)); + } + @Override public void dropView(Session session, QualifiedObjectName viewName) { diff --git a/presto-main/src/main/java/com/facebook/presto/security/AccessControlManager.java b/presto-main/src/main/java/com/facebook/presto/security/AccessControlManager.java index d4d0125959c5..468d7d0ed2f3 100644 --- a/presto-main/src/main/java/com/facebook/presto/security/AccessControlManager.java +++ b/presto-main/src/main/java/com/facebook/presto/security/AccessControlManager.java @@ -514,6 +514,23 @@ public void checkCanCreateView(TransactionId transactionId, Identity identity, A } } + @Override + public void checkCanRenameView(TransactionId transactionId, Identity identity, AccessControlContext context, QualifiedObjectName viewName, QualifiedObjectName newViewName) + { + requireNonNull(context, "context is null"); + requireNonNull(viewName, "viewName is null"); + requireNonNull(newViewName, "newViewName is null"); + + authenticationCheck(() -> checkCanAccessCatalog(identity, context, viewName.getCatalogName())); + + authorizationCheck(() -> systemAccessControl.get().checkCanRenameView(identity, context, toCatalogSchemaTableName(viewName), toCatalogSchemaTableName(newViewName))); + + CatalogAccessControlEntry entry = getConnectorAccessControl(transactionId, viewName.getCatalogName()); + if (entry != null) { + authorizationCheck(() -> entry.getAccessControl().checkCanRenameView(entry.getTransactionHandle(transactionId), identity.toConnectorIdentity(viewName.getCatalogName()), context, toSchemaTableName(viewName), toSchemaTableName(newViewName))); + } + } + @Override public void checkCanDropView(TransactionId transactionId, Identity identity, AccessControlContext context, QualifiedObjectName viewName) { diff --git a/presto-main/src/main/java/com/facebook/presto/security/AllowAllSystemAccessControl.java b/presto-main/src/main/java/com/facebook/presto/security/AllowAllSystemAccessControl.java index 4197801047f7..a0e1ea6c315f 100644 --- a/presto-main/src/main/java/com/facebook/presto/security/AllowAllSystemAccessControl.java +++ b/presto-main/src/main/java/com/facebook/presto/security/AllowAllSystemAccessControl.java @@ -192,6 +192,11 @@ public void checkCanCreateView(Identity identity, AccessControlContext context, { } + @Override + public void checkCanRenameView(Identity identity, AccessControlContext context, CatalogSchemaTableName view, CatalogSchemaTableName newView) + { + } + @Override public void checkCanDropView(Identity identity, AccessControlContext context, CatalogSchemaTableName view) { diff --git a/presto-main/src/main/java/com/facebook/presto/security/FileBasedSystemAccessControl.java b/presto-main/src/main/java/com/facebook/presto/security/FileBasedSystemAccessControl.java index f9bf1d5b976d..199490f096f2 100644 --- a/presto-main/src/main/java/com/facebook/presto/security/FileBasedSystemAccessControl.java +++ b/presto-main/src/main/java/com/facebook/presto/security/FileBasedSystemAccessControl.java @@ -65,6 +65,7 @@ import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameColumn; import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameSchema; import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameTable; +import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameView; import static com.facebook.presto.spi.security.AccessDeniedException.denyRevokeTablePrivilege; import static com.facebook.presto.spi.security.AccessDeniedException.denySetTableProperties; import static com.facebook.presto.spi.security.AccessDeniedException.denySetUser; @@ -388,6 +389,14 @@ public void checkCanCreateView(Identity identity, AccessControlContext context, } } + @Override + public void checkCanRenameView(Identity identity, AccessControlContext context, CatalogSchemaTableName view, CatalogSchemaTableName newView) + { + if (!canAccessCatalog(identity, view.getCatalogName(), ALL)) { + denyRenameView(view.toString(), newView.toString()); + } + } + @Override public void checkCanDropView(Identity identity, AccessControlContext context, CatalogSchemaTableName view) { diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/StatementAnalyzer.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/StatementAnalyzer.java index d9fb6707c20d..1e4e86af0644 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/StatementAnalyzer.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/StatementAnalyzer.java @@ -136,6 +136,7 @@ import com.facebook.presto.sql.tree.RenameColumn; import com.facebook.presto.sql.tree.RenameSchema; import com.facebook.presto.sql.tree.RenameTable; +import com.facebook.presto.sql.tree.RenameView; import com.facebook.presto.sql.tree.ResetSession; import com.facebook.presto.sql.tree.Return; import com.facebook.presto.sql.tree.Revoke; @@ -1006,6 +1007,12 @@ protected Scope visitAlterColumnNotNull(AlterColumnNotNull node, Optional return createAndAssignScope(node, scope); } + @Override + protected Scope visitRenameView(RenameView node, Optional scope) + { + return createAndAssignScope(node, scope); + } + @Override protected Scope visitDropView(DropView node, Optional scope) { diff --git a/presto-main/src/main/java/com/facebook/presto/testing/LocalQueryRunner.java b/presto-main/src/main/java/com/facebook/presto/testing/LocalQueryRunner.java index cefcd193422d..7bf878e0b481 100644 --- a/presto-main/src/main/java/com/facebook/presto/testing/LocalQueryRunner.java +++ b/presto-main/src/main/java/com/facebook/presto/testing/LocalQueryRunner.java @@ -70,6 +70,7 @@ import com.facebook.presto.execution.QueryManagerConfig; import com.facebook.presto.execution.RenameColumnTask; import com.facebook.presto.execution.RenameTableTask; +import com.facebook.presto.execution.RenameViewTask; import com.facebook.presto.execution.ResetSessionTask; import com.facebook.presto.execution.RollbackTask; import com.facebook.presto.execution.ScheduledSplit; @@ -210,6 +211,7 @@ import com.facebook.presto.sql.tree.Prepare; import com.facebook.presto.sql.tree.RenameColumn; import com.facebook.presto.sql.tree.RenameTable; +import com.facebook.presto.sql.tree.RenameView; import com.facebook.presto.sql.tree.ResetSession; import com.facebook.presto.sql.tree.Rollback; import com.facebook.presto.sql.tree.SetProperties; @@ -580,6 +582,7 @@ private LocalQueryRunner(Session defaultSession, FeaturesConfig featuresConfig, .put(DropMaterializedView.class, new DropMaterializedViewTask()) .put(RenameColumn.class, new RenameColumnTask()) .put(RenameTable.class, new RenameTableTask()) + .put(RenameView.class, new RenameViewTask()) .put(ResetSession.class, new ResetSessionTask()) .put(SetSession.class, new SetSessionTask()) .put(Prepare.class, new PrepareTask(sqlParser)) diff --git a/presto-main/src/main/java/com/facebook/presto/testing/TestingAccessControlManager.java b/presto-main/src/main/java/com/facebook/presto/testing/TestingAccessControlManager.java index 1db74c076d0b..fa2e681c1878 100644 --- a/presto-main/src/main/java/com/facebook/presto/testing/TestingAccessControlManager.java +++ b/presto-main/src/main/java/com/facebook/presto/testing/TestingAccessControlManager.java @@ -50,6 +50,7 @@ import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameColumn; import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameSchema; import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameTable; +import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameView; import static com.facebook.presto.spi.security.AccessDeniedException.denySelectColumns; import static com.facebook.presto.spi.security.AccessDeniedException.denySetCatalogSessionProperty; import static com.facebook.presto.spi.security.AccessDeniedException.denySetSystemSessionProperty; @@ -73,6 +74,7 @@ import static com.facebook.presto.testing.TestingAccessControlManager.TestingPrivilegeType.RENAME_COLUMN; import static com.facebook.presto.testing.TestingAccessControlManager.TestingPrivilegeType.RENAME_SCHEMA; import static com.facebook.presto.testing.TestingAccessControlManager.TestingPrivilegeType.RENAME_TABLE; +import static com.facebook.presto.testing.TestingAccessControlManager.TestingPrivilegeType.RENAME_VIEW; import static com.facebook.presto.testing.TestingAccessControlManager.TestingPrivilegeType.SELECT_COLUMN; import static com.facebook.presto.testing.TestingAccessControlManager.TestingPrivilegeType.SET_SESSION; import static com.facebook.presto.testing.TestingAccessControlManager.TestingPrivilegeType.SET_TABLE_PROPERTIES; @@ -286,6 +288,17 @@ public void checkCanCreateView(TransactionId transactionId, Identity identity, A } } + @Override + public void checkCanRenameView(TransactionId transactionId, Identity identity, AccessControlContext context, QualifiedObjectName viewName, QualifiedObjectName newViewName) + { + if (shouldDenyPrivilege(identity.getUser(), viewName.getObjectName(), RENAME_VIEW)) { + denyRenameView(viewName.toString(), newViewName.toString()); + } + if (denyPrivileges.isEmpty()) { + super.checkCanRenameView(transactionId, identity, context, viewName, newViewName); + } + } + @Override public void checkCanDropView(TransactionId transactionId, Identity identity, AccessControlContext context, QualifiedObjectName viewName) { @@ -383,7 +396,7 @@ public enum TestingPrivilegeType CREATE_TABLE, DROP_TABLE, RENAME_TABLE, INSERT_TABLE, DELETE_TABLE, TRUNCATE_TABLE, UPDATE_TABLE, ADD_COLUMN, DROP_COLUMN, RENAME_COLUMN, SELECT_COLUMN, ADD_CONSTRAINT, DROP_CONSTRAINT, - CREATE_VIEW, DROP_VIEW, CREATE_VIEW_WITH_SELECT_COLUMNS, SET_TABLE_PROPERTIES, + CREATE_VIEW, RENAME_VIEW, DROP_VIEW, CREATE_VIEW_WITH_SELECT_COLUMNS, SET_TABLE_PROPERTIES, SET_SESSION } diff --git a/presto-main/src/main/java/com/facebook/presto/util/PrestoDataDefBindingHelper.java b/presto-main/src/main/java/com/facebook/presto/util/PrestoDataDefBindingHelper.java index f83e282373e0..3170b0f62d79 100644 --- a/presto-main/src/main/java/com/facebook/presto/util/PrestoDataDefBindingHelper.java +++ b/presto-main/src/main/java/com/facebook/presto/util/PrestoDataDefBindingHelper.java @@ -42,6 +42,7 @@ import com.facebook.presto.execution.RenameColumnTask; import com.facebook.presto.execution.RenameSchemaTask; import com.facebook.presto.execution.RenameTableTask; +import com.facebook.presto.execution.RenameViewTask; import com.facebook.presto.execution.ResetSessionTask; import com.facebook.presto.execution.RevokeRolesTask; import com.facebook.presto.execution.RevokeTask; @@ -80,6 +81,7 @@ import com.facebook.presto.sql.tree.RenameColumn; import com.facebook.presto.sql.tree.RenameSchema; import com.facebook.presto.sql.tree.RenameTable; +import com.facebook.presto.sql.tree.RenameView; import com.facebook.presto.sql.tree.ResetSession; import com.facebook.presto.sql.tree.Revoke; import com.facebook.presto.sql.tree.RevokeRoles; @@ -129,6 +131,7 @@ private PrestoDataDefBindingHelper() {} dataDefBuilder.put(DropTable.class, DropTableTask.class); dataDefBuilder.put(TruncateTable.class, TruncateTableTask.class); dataDefBuilder.put(CreateView.class, CreateViewTask.class); + dataDefBuilder.put(RenameView.class, RenameViewTask.class); dataDefBuilder.put(DropView.class, DropViewTask.class); dataDefBuilder.put(CreateMaterializedView.class, CreateMaterializedViewTask.class); dataDefBuilder.put(DropMaterializedView.class, DropMaterializedViewTask.class); diff --git a/presto-main/src/test/java/com/facebook/presto/metadata/AbstractMockMetadata.java b/presto-main/src/test/java/com/facebook/presto/metadata/AbstractMockMetadata.java index 48210df7d559..49a6051c2236 100644 --- a/presto-main/src/test/java/com/facebook/presto/metadata/AbstractMockMetadata.java +++ b/presto-main/src/test/java/com/facebook/presto/metadata/AbstractMockMetadata.java @@ -464,6 +464,12 @@ public void createView(Session session, String catalogName, ConnectorTableMetada throw new UnsupportedOperationException(); } + @Override + public void renameView(Session session, QualifiedObjectName source, QualifiedObjectName target) + { + throw new UnsupportedOperationException(); + } + @Override public void dropView(Session session, QualifiedObjectName viewName) { diff --git a/presto-main/src/test/java/com/facebook/presto/security/TestAccessControlManager.java b/presto-main/src/test/java/com/facebook/presto/security/TestAccessControlManager.java index 7039ba042697..767e0a06d35b 100644 --- a/presto-main/src/test/java/com/facebook/presto/security/TestAccessControlManager.java +++ b/presto-main/src/test/java/com/facebook/presto/security/TestAccessControlManager.java @@ -458,6 +458,12 @@ public void checkCanCreateView(ConnectorTransactionHandle transactionHandle, Con throw new UnsupportedOperationException(); } + @Override + public void checkCanRenameView(ConnectorTransactionHandle transactionHandle, ConnectorIdentity identity, AccessControlContext context, SchemaTableName viewName, SchemaTableName newViewName) + { + throw new UnsupportedOperationException(); + } + @Override public void checkCanDropView(ConnectorTransactionHandle transactionHandle, ConnectorIdentity identity, AccessControlContext context, SchemaTableName viewName) { diff --git a/presto-parser/src/main/antlr4/com/facebook/presto/sql/parser/SqlBase.g4 b/presto-parser/src/main/antlr4/com/facebook/presto/sql/parser/SqlBase.g4 index c5b80d73eb2b..e5ee6b44b9a4 100644 --- a/presto-parser/src/main/antlr4/com/facebook/presto/sql/parser/SqlBase.g4 +++ b/presto-parser/src/main/antlr4/com/facebook/presto/sql/parser/SqlBase.g4 @@ -74,6 +74,8 @@ statement | type) #createType | CREATE (OR REPLACE)? VIEW qualifiedName (SECURITY (DEFINER | INVOKER))? AS query #createView + | ALTER VIEW (IF EXISTS)? from=qualifiedName + RENAME TO to=qualifiedName #renameView | DROP VIEW (IF EXISTS)? qualifiedName #dropView | CREATE MATERIALIZED VIEW (IF NOT EXISTS)? qualifiedName (COMMENT string)? diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/SqlFormatter.java b/presto-parser/src/main/java/com/facebook/presto/sql/SqlFormatter.java index ea7c192cc859..bb853ba01a8e 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/SqlFormatter.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/SqlFormatter.java @@ -82,6 +82,7 @@ import com.facebook.presto.sql.tree.RenameColumn; import com.facebook.presto.sql.tree.RenameSchema; import com.facebook.presto.sql.tree.RenameTable; +import com.facebook.presto.sql.tree.RenameView; import com.facebook.presto.sql.tree.ResetSession; import com.facebook.presto.sql.tree.Return; import com.facebook.presto.sql.tree.Revoke; @@ -689,6 +690,20 @@ protected Void visitExternalBodyReference(ExternalBodyReference node, Integer in return null; } + @Override + protected Void visitRenameView(RenameView node, Integer context) + { + builder.append("ALTER VIEW "); + if (node.isExists()) { + builder.append("IF EXISTS "); + } + builder.append(formatName(node.getSource())) + .append(" RENAME TO ") + .append(formatName(node.getTarget())); + + return null; + } + @Override protected Void visitDropView(DropView node, Integer context) { diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/parser/AstBuilder.java b/presto-parser/src/main/java/com/facebook/presto/sql/parser/AstBuilder.java index f916feed3ad7..7b1dd90a57f5 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/parser/AstBuilder.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/parser/AstBuilder.java @@ -127,6 +127,7 @@ import com.facebook.presto.sql.tree.RenameColumn; import com.facebook.presto.sql.tree.RenameSchema; import com.facebook.presto.sql.tree.RenameTable; +import com.facebook.presto.sql.tree.RenameView; import com.facebook.presto.sql.tree.ResetSession; import com.facebook.presto.sql.tree.Return; import com.facebook.presto.sql.tree.Revoke; @@ -486,6 +487,12 @@ public Node visitRenameColumn(SqlBaseParser.RenameColumnContext context) context.EXISTS().stream().anyMatch(node -> node.getSymbol().getTokenIndex() > context.COLUMN().getSymbol().getTokenIndex())); } + @Override + public Node visitRenameView(SqlBaseParser.RenameViewContext context) + { + return new RenameView(getLocation(context), getQualifiedName(context.from), getQualifiedName(context.to), context.EXISTS() != null); + } + @Override public Node visitAnalyze(SqlBaseParser.AnalyzeContext context) { diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/AstVisitor.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/AstVisitor.java index 8eec8cf4936f..8f42707192a4 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/tree/AstVisitor.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/AstVisitor.java @@ -627,6 +627,11 @@ protected R visitCreateView(CreateView node, C context) return visitStatement(node, context); } + protected R visitRenameView(RenameView node, C context) + { + return visitStatement(node, context); + } + protected R visitDropView(DropView node, C context) { return visitStatement(node, context); diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/RenameView.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/RenameView.java new file mode 100644 index 000000000000..f591d733ed85 --- /dev/null +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/RenameView.java @@ -0,0 +1,108 @@ +/* + * 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.tree; + +import com.google.common.collect.ImmutableList; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +public final class RenameView + extends Statement +{ + private final QualifiedName source; + private final QualifiedName target; + + private final boolean exists; + + public RenameView(QualifiedName source, QualifiedName target, boolean exists) + { + this(Optional.empty(), source, target, exists); + } + + public RenameView(NodeLocation location, QualifiedName source, QualifiedName target, boolean exists) + { + this(Optional.of(location), source, target, exists); + } + + private RenameView(Optional location, QualifiedName source, QualifiedName target, boolean exists) + { + super(location); + this.source = requireNonNull(source, "source name is null"); + this.target = requireNonNull(target, "target name is null"); + this.exists = exists; + } + + public QualifiedName getSource() + { + return source; + } + + public QualifiedName getTarget() + { + return target; + } + + public boolean isExists() + { + return exists; + } + + @Override + public R accept(AstVisitor visitor, C context) + { + return visitor.visitRenameView(this, context); + } + + @Override + public List getChildren() + { + return ImmutableList.of(); + } + + @Override + public int hashCode() + { + return Objects.hash(source, target, exists); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if ((obj == null) || (getClass() != obj.getClass())) { + return false; + } + RenameView o = (RenameView) obj; + return Objects.equals(source, o.source) && + Objects.equals(target, o.target) && + Objects.equals(exists, o.exists); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("source", source) + .add("target", target) + .add("exists", exists) + .toString(); + } +} diff --git a/presto-parser/src/test/java/com/facebook/presto/sql/parser/TestSqlParser.java b/presto-parser/src/test/java/com/facebook/presto/sql/parser/TestSqlParser.java index ce3715b8c296..68b590834653 100644 --- a/presto-parser/src/test/java/com/facebook/presto/sql/parser/TestSqlParser.java +++ b/presto-parser/src/test/java/com/facebook/presto/sql/parser/TestSqlParser.java @@ -110,6 +110,7 @@ import com.facebook.presto.sql.tree.RenameColumn; import com.facebook.presto.sql.tree.RenameSchema; import com.facebook.presto.sql.tree.RenameTable; +import com.facebook.presto.sql.tree.RenameView; import com.facebook.presto.sql.tree.ResetSession; import com.facebook.presto.sql.tree.Return; import com.facebook.presto.sql.tree.Revoke; @@ -1505,6 +1506,13 @@ public void testTruncateTable() assertStatement("TRUNCATE TABLE a.b.c", new TruncateTable(QualifiedName.of("a", "b", "c"))); } + @Test + public void testRenameView() + { + assertStatement("ALTER VIEW a RENAME TO b", new RenameView(QualifiedName.of("a"), QualifiedName.of("b"), false)); + assertStatement("ALTER VIEW IF EXISTS a RENAME TO b", new RenameView(QualifiedName.of("a"), QualifiedName.of("b"), true)); + } + @Test public void testDropView() { diff --git a/presto-plugin-toolkit/src/main/java/com/facebook/presto/plugin/base/security/AllowAllAccessControl.java b/presto-plugin-toolkit/src/main/java/com/facebook/presto/plugin/base/security/AllowAllAccessControl.java index 38f94530bedd..30e99555fe43 100644 --- a/presto-plugin-toolkit/src/main/java/com/facebook/presto/plugin/base/security/AllowAllAccessControl.java +++ b/presto-plugin-toolkit/src/main/java/com/facebook/presto/plugin/base/security/AllowAllAccessControl.java @@ -131,6 +131,11 @@ public void checkCanCreateView(ConnectorTransactionHandle transaction, Connector { } + @Override + public void checkCanRenameView(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName viewName, SchemaTableName newViewName) + { + } + @Override public void checkCanDropView(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName viewName) { diff --git a/presto-plugin-toolkit/src/main/java/com/facebook/presto/plugin/base/security/FileBasedAccessControl.java b/presto-plugin-toolkit/src/main/java/com/facebook/presto/plugin/base/security/FileBasedAccessControl.java index d9d13ccdb314..e1ee1b874815 100644 --- a/presto-plugin-toolkit/src/main/java/com/facebook/presto/plugin/base/security/FileBasedAccessControl.java +++ b/presto-plugin-toolkit/src/main/java/com/facebook/presto/plugin/base/security/FileBasedAccessControl.java @@ -57,6 +57,7 @@ import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameColumn; import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameSchema; import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameTable; +import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameView; import static com.facebook.presto.spi.security.AccessDeniedException.denyRevokeTablePrivilege; import static com.facebook.presto.spi.security.AccessDeniedException.denySelectTable; import static com.facebook.presto.spi.security.AccessDeniedException.denySetTableProperties; @@ -227,6 +228,14 @@ public void checkCanCreateView(ConnectorTransactionHandle transaction, Connector } } + @Override + public void checkCanRenameView(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName viewName, SchemaTableName newViewName) + { + if (!checkTablePermission(identity, viewName, OWNERSHIP) || !checkTablePermission(identity, newViewName, OWNERSHIP)) { + denyRenameView(viewName.toString(), newViewName.toString()); + } + } + @Override public void checkCanDropView(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName viewName) { diff --git a/presto-plugin-toolkit/src/main/java/com/facebook/presto/plugin/base/security/ForwardingConnectorAccessControl.java b/presto-plugin-toolkit/src/main/java/com/facebook/presto/plugin/base/security/ForwardingConnectorAccessControl.java index bb4f74134f02..dd28ee9183d5 100644 --- a/presto-plugin-toolkit/src/main/java/com/facebook/presto/plugin/base/security/ForwardingConnectorAccessControl.java +++ b/presto-plugin-toolkit/src/main/java/com/facebook/presto/plugin/base/security/ForwardingConnectorAccessControl.java @@ -167,6 +167,12 @@ public void checkCanCreateView(ConnectorTransactionHandle transactionHandle, Con delegate().checkCanCreateView(transactionHandle, identity, context, viewName); } + @Override + public void checkCanRenameView(ConnectorTransactionHandle transactionHandle, ConnectorIdentity identity, AccessControlContext context, SchemaTableName viewName, SchemaTableName newViewName) + { + delegate().checkCanRenameView(transactionHandle, identity, context, viewName, newViewName); + } + @Override public void checkCanDropView(ConnectorTransactionHandle transactionHandle, ConnectorIdentity identity, AccessControlContext context, SchemaTableName viewName) { diff --git a/presto-plugin-toolkit/src/main/java/com/facebook/presto/plugin/base/security/ForwardingSystemAccessControl.java b/presto-plugin-toolkit/src/main/java/com/facebook/presto/plugin/base/security/ForwardingSystemAccessControl.java index 0adec4a48aa3..6e9cd6a91c71 100644 --- a/presto-plugin-toolkit/src/main/java/com/facebook/presto/plugin/base/security/ForwardingSystemAccessControl.java +++ b/presto-plugin-toolkit/src/main/java/com/facebook/presto/plugin/base/security/ForwardingSystemAccessControl.java @@ -206,6 +206,12 @@ public void checkCanCreateView(Identity identity, AccessControlContext context, delegate().checkCanCreateView(identity, context, view); } + @Override + public void checkCanRenameView(Identity identity, AccessControlContext context, CatalogSchemaTableName view, CatalogSchemaTableName newView) + { + delegate().checkCanRenameView(identity, context, view, newView); + } + @Override public void checkCanDropView(Identity identity, AccessControlContext context, CatalogSchemaTableName view) { diff --git a/presto-plugin-toolkit/src/main/java/com/facebook/presto/plugin/base/security/ReadOnlyAccessControl.java b/presto-plugin-toolkit/src/main/java/com/facebook/presto/plugin/base/security/ReadOnlyAccessControl.java index def67490ab69..30022c207985 100644 --- a/presto-plugin-toolkit/src/main/java/com/facebook/presto/plugin/base/security/ReadOnlyAccessControl.java +++ b/presto-plugin-toolkit/src/main/java/com/facebook/presto/plugin/base/security/ReadOnlyAccessControl.java @@ -25,6 +25,7 @@ import java.util.Set; import static com.facebook.presto.spi.security.AccessDeniedException.denyGrantTablePrivilege; +import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameView; import static com.facebook.presto.spi.security.AccessDeniedException.denyRevokeTablePrivilege; import static com.facebook.presto.spi.security.AccessDeniedException.denyTruncateTable; @@ -71,6 +72,12 @@ public void checkCanCreateViewWithSelectFromColumns(ConnectorTransactionHandle t // allow } + @Override + public void checkCanRenameView(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName viewName, SchemaTableName newViewName) + { + denyRenameView(viewName.toString(), newViewName.toString()); + } + @Override public void checkCanSetCatalogSessionProperty(ConnectorTransactionHandle transactionHandle, ConnectorIdentity identity, AccessControlContext context, String propertyName) { diff --git a/presto-plugin-toolkit/src/test/java/com/facebook/presto/plugin/base/security/TestFileBasedAccessControl.java b/presto-plugin-toolkit/src/test/java/com/facebook/presto/plugin/base/security/TestFileBasedAccessControl.java index 1c081887df65..daf64d750d42 100644 --- a/presto-plugin-toolkit/src/test/java/com/facebook/presto/plugin/base/security/TestFileBasedAccessControl.java +++ b/presto-plugin-toolkit/src/test/java/com/facebook/presto/plugin/base/security/TestFileBasedAccessControl.java @@ -69,7 +69,12 @@ public void testTableRules() accessControl.checkCanDeleteFromTable(TRANSACTION_HANDLE, user("bob"), CONTEXT, new SchemaTableName("bobschema", "bobtable")); accessControl.checkCanSelectFromColumns(TRANSACTION_HANDLE, user("joe"), CONTEXT, new SchemaTableName("bobschema", "bobtable"), ImmutableSet.of()); accessControl.checkCanCreateViewWithSelectFromColumns(TRANSACTION_HANDLE, user("bob"), CONTEXT, new SchemaTableName("bobschema", "bobtable"), ImmutableSet.of()); + accessControl.checkCanRenameTable(TRANSACTION_HANDLE, user("admin"), CONTEXT, new SchemaTableName("bobschema", "bobtable"), new SchemaTableName("aliceschema", "newbobtable")); + accessControl.checkCanRenameTable(TRANSACTION_HANDLE, user("alice"), CONTEXT, new SchemaTableName("aliceschema", "alicetable"), new SchemaTableName("aliceschema", "newalicetable")); + accessControl.checkCanRenameView(TRANSACTION_HANDLE, user("admin"), CONTEXT, new SchemaTableName("bobschema", "bobview"), new SchemaTableName("aliceschema", "newbobview")); + accessControl.checkCanRenameView(TRANSACTION_HANDLE, user("alice"), CONTEXT, new SchemaTableName("aliceschema", "aliceview"), new SchemaTableName("aliceschema", "newaliceview")); accessControl.checkCanDropTable(TRANSACTION_HANDLE, user("admin"), CONTEXT, new SchemaTableName("bobschema", "bobtable")); + assertDenied(() -> accessControl.checkCanRenameTable(TRANSACTION_HANDLE, user("bob"), CONTEXT, new SchemaTableName("bobschema", "bobtable"), new SchemaTableName("bobschema", "newbobtable"))); accessControl.checkCanSetTableProperties(TRANSACTION_HANDLE, user("admin"), CONTEXT, new SchemaTableName("bobschema", "bobtable"), ImmutableMap.of()); accessControl.checkCanSetTableProperties(TRANSACTION_HANDLE, user("alice"), CONTEXT, new SchemaTableName("aliceSchema", "aliceTable"), ImmutableMap.of()); assertDenied(() -> accessControl.checkCanInsertIntoTable(TRANSACTION_HANDLE, user("alice"), CONTEXT, new SchemaTableName("bobschema", "bobtable"))); @@ -79,6 +84,8 @@ public void testTableRules() assertDenied(() -> accessControl.checkCanSelectFromColumns(TRANSACTION_HANDLE, user("admin"), CONTEXT, new SchemaTableName("secret", "secret"), ImmutableSet.of())); assertDenied(() -> accessControl.checkCanSelectFromColumns(TRANSACTION_HANDLE, user("joe"), CONTEXT, new SchemaTableName("secret", "secret"), ImmutableSet.of())); assertDenied(() -> accessControl.checkCanCreateViewWithSelectFromColumns(TRANSACTION_HANDLE, user("joe"), CONTEXT, new SchemaTableName("bobschema", "bobtable"), ImmutableSet.of())); + assertDenied(() -> accessControl.checkCanRenameView(TRANSACTION_HANDLE, user("bob"), CONTEXT, new SchemaTableName("bobschema", "bobview"), new SchemaTableName("bobschema", "newbobview"))); + assertDenied(() -> accessControl.checkCanRenameView(TRANSACTION_HANDLE, user("alice"), CONTEXT, new SchemaTableName("aliceschema", "alicetable"), new SchemaTableName("bobschema", "newalicetable"))); } @Test diff --git a/presto-plugin-toolkit/src/test/resources/table.json b/presto-plugin-toolkit/src/test/resources/table.json index 18b76320fc16..2ebef52bdcdb 100644 --- a/presto-plugin-toolkit/src/test/resources/table.json +++ b/presto-plugin-toolkit/src/test/resources/table.json @@ -9,6 +9,11 @@ "schema": ".*", "privileges": ["SELECT", "INSERT", "DELETE", "OWNERSHIP"] }, + { + "user": "alice", + "schema": "aliceschema", + "privileges": ["SELECT", "INSERT", "DELETE", "OWNERSHIP"] + }, { "user": "bob", "schema": "bobschema", diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/connector/ConnectorAccessControl.java b/presto-spi/src/main/java/com/facebook/presto/spi/connector/ConnectorAccessControl.java index 62f7208301ee..e5621d458c0b 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/connector/ConnectorAccessControl.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/connector/ConnectorAccessControl.java @@ -44,6 +44,7 @@ import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameColumn; import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameSchema; import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameTable; +import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameView; import static com.facebook.presto.spi.security.AccessDeniedException.denyRevokeRoles; import static com.facebook.presto.spi.security.AccessDeniedException.denyRevokeTablePrivilege; import static com.facebook.presto.spi.security.AccessDeniedException.denySelectColumns; @@ -270,6 +271,16 @@ default void checkCanCreateView(ConnectorTransactionHandle transactionHandle, Co denyCreateView(viewName.toString()); } + /** + * Check if identity is allowed to rename the specified view in this catalog. + * + * @throws com.facebook.presto.spi.security.AccessDeniedException if not allowed + */ + default void checkCanRenameView(ConnectorTransactionHandle transactionHandle, ConnectorIdentity identity, AccessControlContext context, SchemaTableName viewName, SchemaTableName newViewName) + { + denyRenameView(viewName.toString(), newViewName.toString()); + } + /** * Check if identity is allowed to drop the specified view in this catalog. * diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/connector/ConnectorMetadata.java b/presto-spi/src/main/java/com/facebook/presto/spi/connector/ConnectorMetadata.java index 06ebb4acbc12..d31c12b68e23 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/connector/ConnectorMetadata.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/connector/ConnectorMetadata.java @@ -561,6 +561,14 @@ default void createView(ConnectorSession session, ConnectorTableMetadata viewMet throw new PrestoException(NOT_SUPPORTED, "This connector does not support creating views"); } + /** + * Rename the specified view + */ + default void renameView(ConnectorSession session, SchemaTableName viewName, SchemaTableName newViewName) + { + throw new PrestoException(NOT_SUPPORTED, "This connector does not support renaming views"); + } + /** * Drop the specified view. */ diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/connector/classloader/ClassLoaderSafeConnectorMetadata.java b/presto-spi/src/main/java/com/facebook/presto/spi/connector/classloader/ClassLoaderSafeConnectorMetadata.java index f1e540717614..fa1aeb780f69 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/connector/classloader/ClassLoaderSafeConnectorMetadata.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/connector/classloader/ClassLoaderSafeConnectorMetadata.java @@ -474,6 +474,14 @@ public void createView(ConnectorSession session, ConnectorTableMetadata viewMeta } } + @Override + public void renameView(ConnectorSession session, SchemaTableName viewName, SchemaTableName newViewName) + { + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { + delegate.renameView(session, viewName, newViewName); + } + } + @Override public ColumnHandle getDeleteRowIdColumnHandle(ConnectorSession session, ConnectorTableHandle tableHandle) { diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/security/AccessControl.java b/presto-spi/src/main/java/com/facebook/presto/spi/security/AccessControl.java index c75f3ee58e24..131c0af1577d 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/security/AccessControl.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/security/AccessControl.java @@ -193,6 +193,13 @@ default AuthorizedIdentity selectAuthorizedIdentity(Identity identity, AccessCon */ void checkCanCreateView(TransactionId transactionId, Identity identity, AccessControlContext context, QualifiedObjectName viewName); + /** + * Check if identity is allowed to rename the specified view. + * + * @throws com.facebook.presto.spi.security.AccessDeniedException if not allowed + */ + void checkCanRenameView(TransactionId transactionId, Identity identity, AccessControlContext context, QualifiedObjectName viewName, QualifiedObjectName newViewName); + /** * Check if identity is allowed to drop the specified view. * diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/security/AccessDeniedException.java b/presto-spi/src/main/java/com/facebook/presto/spi/security/AccessDeniedException.java index 1766e1b96199..5992554dd887 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/security/AccessDeniedException.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/security/AccessDeniedException.java @@ -251,6 +251,16 @@ public static void denyCreateViewWithSelect(String sourceName, ConnectorIdentity throw new AccessDeniedException(format("View owner '%s' cannot create view that selects from %s%s", identity.getUser(), sourceName, formatExtraInfo(extraInfo))); } + public static void denyRenameView(String viewName, String newViewName) + { + denyRenameView(viewName, newViewName, null); + } + + public static void denyRenameView(String viewName, String newViewName, String extraInfo) + { + throw new AccessDeniedException(format("Cannot rename view from %s to %s%s", viewName, newViewName, formatExtraInfo(extraInfo))); + } + public static void denyDropView(String viewName) { denyDropView(viewName, null); diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/security/AllowAllAccessControl.java b/presto-spi/src/main/java/com/facebook/presto/spi/security/AllowAllAccessControl.java index 70458ba83ef0..f07a4e71abc1 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/security/AllowAllAccessControl.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/security/AllowAllAccessControl.java @@ -145,6 +145,11 @@ public void checkCanCreateView(TransactionId transactionId, Identity identity, A { } + @Override + public void checkCanRenameView(TransactionId transactionId, Identity identity, AccessControlContext context, QualifiedObjectName viewName, QualifiedObjectName newViewName) + { + } + @Override public void checkCanDropView(TransactionId transactionId, Identity identity, AccessControlContext context, QualifiedObjectName viewName) { diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/security/DenyAllAccessControl.java b/presto-spi/src/main/java/com/facebook/presto/spi/security/DenyAllAccessControl.java index 0fdbbe5576b6..72b4dbabfc84 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/security/DenyAllAccessControl.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/security/DenyAllAccessControl.java @@ -47,6 +47,7 @@ import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameColumn; import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameSchema; import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameTable; +import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameView; import static com.facebook.presto.spi.security.AccessDeniedException.denyRevokeRoles; import static com.facebook.presto.spi.security.AccessDeniedException.denyRevokeTablePrivilege; import static com.facebook.presto.spi.security.AccessDeniedException.denySelectColumns; @@ -207,6 +208,12 @@ public void checkCanCreateView(TransactionId transactionId, Identity identity, A denyCreateView(viewName.toString()); } + @Override + public void checkCanRenameView(TransactionId transactionId, Identity identity, AccessControlContext context, QualifiedObjectName viewName, QualifiedObjectName newViewName) + { + denyRenameView(viewName.toString(), newViewName.toString()); + } + @Override public void checkCanDropView(TransactionId transactionId, Identity identity, AccessControlContext context, QualifiedObjectName viewName) { diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/security/SystemAccessControl.java b/presto-spi/src/main/java/com/facebook/presto/spi/security/SystemAccessControl.java index 7d209ce6b392..fc5e8cbf6805 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/security/SystemAccessControl.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/security/SystemAccessControl.java @@ -42,6 +42,7 @@ import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameColumn; import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameSchema; import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameTable; +import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameView; import static com.facebook.presto.spi.security.AccessDeniedException.denyRevokeTablePrivilege; import static com.facebook.presto.spi.security.AccessDeniedException.denySelectColumns; import static com.facebook.presto.spi.security.AccessDeniedException.denySetCatalogSessionProperty; @@ -300,6 +301,16 @@ default void checkCanCreateView(Identity identity, AccessControlContext context, denyCreateView(view.toString()); } + /** + * Check if identity is allowed to rename the specified view in a catalog. + * + * @throws AccessDeniedException if not allowed + */ + default void checkCanRenameView(Identity identity, AccessControlContext context, CatalogSchemaTableName view, CatalogSchemaTableName newView) + { + denyRenameView(view.toString(), newView.toString()); + } + /** * Check if identity is allowed to drop the specified view in a catalog. * From 23ed19074cdb2c057f44715a708e273a0c8a0c0e Mon Sep 17 00:00:00 2001 From: Deepa-George Date: Wed, 13 Nov 2024 14:49:49 +0530 Subject: [PATCH 11/25] Add support for ALTER VIEW [ IF EXISTS ] RENAME TO statement in Iceberg connector --- .../src/main/sphinx/connector/iceberg.rst | 9 ++++++++ .../metastore/file/FileHiveMetastore.java | 1 + .../presto/iceberg/IcebergHiveMetadata.java | 7 +++++++ .../iceberg/TestIcebergMetadataListing.java | 21 +++++++++++++++++++ 4 files changed, 38 insertions(+) diff --git a/presto-docs/src/main/sphinx/connector/iceberg.rst b/presto-docs/src/main/sphinx/connector/iceberg.rst index 504a37d31dfb..919c75d995e2 100644 --- a/presto-docs/src/main/sphinx/connector/iceberg.rst +++ b/presto-docs/src/main/sphinx/connector/iceberg.rst @@ -1214,6 +1214,15 @@ For example, to set `commit_retries` to 6 for the table `iceberg.web.page_views_ ALTER TABLE iceberg.web.page_views_v2 SET PROPERTIES (commit_retries = 6); +ALTER VIEW +^^^^^^^^^^ + +Alter view operations to alter the name of an existing view to a new name is supported in the Iceberg connector. + +.. code-block:: sql + + ALTER VIEW iceberg.web.page_views RENAME TO iceberg.web.page_new_views; + TRUNCATE ^^^^^^^^ diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/file/FileHiveMetastore.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/file/FileHiveMetastore.java index 5556948a9b35..560bf7dd8f84 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/file/FileHiveMetastore.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/file/FileHiveMetastore.java @@ -99,6 +99,7 @@ import static com.facebook.presto.hive.metastore.MetastoreUtil.getHiveBasicStatistics; import static com.facebook.presto.hive.metastore.MetastoreUtil.getPartitionNamesWithEmptyVersion; import static com.facebook.presto.hive.metastore.MetastoreUtil.isIcebergTable; +import static com.facebook.presto.hive.metastore.MetastoreUtil.isIcebergView; import static com.facebook.presto.hive.metastore.MetastoreUtil.makePartName; import static com.facebook.presto.hive.metastore.MetastoreUtil.toPartitionValues; import static com.facebook.presto.hive.metastore.MetastoreUtil.updateStatisticsParameters; diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergHiveMetadata.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergHiveMetadata.java index f8fdd1be61ce..f1f3777613a7 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergHiveMetadata.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergHiveMetadata.java @@ -421,6 +421,13 @@ public Map getViews(ConnectorSession s return views.build(); } + @Override + public void renameView(ConnectorSession session, SchemaTableName source, SchemaTableName target) + { + // Not checking if source view exists as this is already done in RenameViewTask + metastore.renameTable(getMetastoreContext(session), source.getSchemaName(), source.getTableName(), target.getSchemaName(), target.getTableName()); + } + @Override public void dropView(ConnectorSession session, SchemaTableName viewName) { diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergMetadataListing.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergMetadataListing.java index 4cb866e1072b..4b18df2cacce 100644 --- a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergMetadataListing.java +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergMetadataListing.java @@ -153,4 +153,25 @@ public void testTableValidation() assertQuerySucceeds("SELECT * FROM iceberg.test_schema.iceberg_table1"); assertQueryFails("SELECT * FROM iceberg.test_schema.hive_table", "Not an Iceberg table: test_schema.hive_table"); } + + @Test + public void testRenameView() + { + assertQuerySucceeds("CREATE TABLE iceberg.test_schema.iceberg_test_table (_string VARCHAR, _integer INTEGER)"); + assertUpdate("CREATE VIEW iceberg.test_schema.test_view_to_be_renamed AS SELECT * FROM iceberg.test_schema.iceberg_test_table"); + assertUpdate("ALTER VIEW IF EXISTS iceberg.test_schema.test_view_to_be_renamed RENAME TO iceberg.test_schema.test_view_renamed"); + assertUpdate("CREATE VIEW iceberg.test_schema.test_view2_to_be_renamed AS SELECT * FROM iceberg.test_schema.iceberg_test_table"); + assertUpdate("ALTER VIEW iceberg.test_schema.test_view2_to_be_renamed RENAME TO iceberg.test_schema.test_view2_renamed"); + assertQuerySucceeds("SELECT * FROM iceberg.test_schema.test_view_renamed"); + assertQuerySucceeds("SELECT * FROM iceberg.test_schema.test_view2_renamed"); + assertUpdate("DROP VIEW iceberg.test_schema.test_view_renamed"); + assertUpdate("DROP VIEW iceberg.test_schema.test_view2_renamed"); + assertUpdate("DROP TABLE iceberg.test_schema.iceberg_test_table"); + } + @Test + public void testRenameViewIfNotExists() + { + assertQueryFails("ALTER VIEW iceberg.test_schema.test_rename_view_not_exist RENAME TO iceberg.test_schema.test_renamed_view_not_exist", "line 1:1: View 'iceberg.test_schema.test_rename_view_not_exist' does not exist"); + assertQuerySucceeds("ALTER VIEW IF EXISTS iceberg.test_schema.test_rename_view_not_exist RENAME TO iceberg.test_schema.test_renamed_view_not_exist"); + } } From 50a742523e210a7d7baa67ceb9c3369b8df61f42 Mon Sep 17 00:00:00 2001 From: Deepa-George Date: Wed, 13 Nov 2024 14:59:56 +0530 Subject: [PATCH 12/25] Add support for ALTER VIEW [ IF EXISTS ] RENAME TO statement in Memory connector --- .../src/main/sphinx/connector/memory.rst | 9 ++++++++ .../presto/plugin/memory/MemoryMetadata.java | 15 +++++++++++++ .../plugin/memory/TestMemoryMetadata.java | 10 ++++++++- .../presto/plugin/memory/TestMemorySmoke.java | 22 +++++++++++++++++++ 4 files changed, 55 insertions(+), 1 deletion(-) diff --git a/presto-docs/src/main/sphinx/connector/memory.rst b/presto-docs/src/main/sphinx/connector/memory.rst index 0a49cc4a6290..041282c60d84 100644 --- a/presto-docs/src/main/sphinx/connector/memory.rst +++ b/presto-docs/src/main/sphinx/connector/memory.rst @@ -92,6 +92,15 @@ To delete an existing table: .. note:: After using ``DROP TABLE``, memory is not released immediately. It is released after the next write access to the memory connector. +ALTER VIEW +^^^^^^^^^^ + +Alter view operations to alter the name of an existing view to a new name is supported in the Memory connector. + +.. code-block:: sql + + ALTER VIEW memory.default.nation RENAME TO memory.default.new_nation; + Memory Connector Limitations ---------------------------- diff --git a/presto-memory/src/main/java/com/facebook/presto/plugin/memory/MemoryMetadata.java b/presto-memory/src/main/java/com/facebook/presto/plugin/memory/MemoryMetadata.java index 0c8c9e8b62f9..b600eabc8ea1 100644 --- a/presto-memory/src/main/java/com/facebook/presto/plugin/memory/MemoryMetadata.java +++ b/presto-memory/src/main/java/com/facebook/presto/plugin/memory/MemoryMetadata.java @@ -291,6 +291,21 @@ else if (views.putIfAbsent(viewName, viewData) != null) { } } + @Override + public synchronized void renameView(ConnectorSession session, SchemaTableName viewName, SchemaTableName newViewName) + { + checkSchemaExists(newViewName.getSchemaName()); + if (tableIds.containsKey(newViewName)) { + throw new PrestoException(ALREADY_EXISTS, "Table already exists: " + newViewName); + } + + if (views.containsKey(newViewName)) { + throw new PrestoException(ALREADY_EXISTS, "View already exists: " + newViewName); + } + + views.put(newViewName, views.remove(viewName)); + } + @Override public synchronized void dropView(ConnectorSession session, SchemaTableName viewName) { diff --git a/presto-memory/src/test/java/com/facebook/presto/plugin/memory/TestMemoryMetadata.java b/presto-memory/src/test/java/com/facebook/presto/plugin/memory/TestMemoryMetadata.java index 1a1c200935f2..acc108c57a76 100644 --- a/presto-memory/src/test/java/com/facebook/presto/plugin/memory/TestMemoryMetadata.java +++ b/presto-memory/src/test/java/com/facebook/presto/plugin/memory/TestMemoryMetadata.java @@ -226,6 +226,8 @@ public void testViews() test2, ImmutableList.of(new ColumnMetadata("a", BIGINT))); + SchemaTableName test3 = new SchemaTableName("test", "test_view3"); + // create schema metadata.createSchema(SESSION, "test", ImmutableMap.of()); @@ -265,8 +267,14 @@ public void testViews() views = metadata.getViews(SESSION, new SchemaTablePrefix("test")); assertEquals(views.keySet(), ImmutableSet.of(test2)); + // rename second view + metadata.renameView(SESSION, test2, test3); + + Map result = metadata.getViews(SESSION, new SchemaTablePrefix("test")); + assertTrue(result.containsKey(test3)); + // drop second view - metadata.dropView(SESSION, test2); + metadata.dropView(SESSION, test3); views = metadata.getViews(SESSION, new SchemaTablePrefix("test")); assertTrue(views.isEmpty()); diff --git a/presto-memory/src/test/java/com/facebook/presto/plugin/memory/TestMemorySmoke.java b/presto-memory/src/test/java/com/facebook/presto/plugin/memory/TestMemorySmoke.java index 35c31c556540..4e0a43685291 100644 --- a/presto-memory/src/test/java/com/facebook/presto/plugin/memory/TestMemorySmoke.java +++ b/presto-memory/src/test/java/com/facebook/presto/plugin/memory/TestMemorySmoke.java @@ -185,6 +185,28 @@ public void testViews() assertQueryFails("DROP VIEW test_view", "line 1:1: View 'memory.default.test_view' does not exist"); } + @Test + public void testRenameView() + { + @Language("SQL") String query = "SELECT orderkey, orderstatus, totalprice / 2 half FROM orders"; + + assertUpdate("CREATE VIEW test_view_to_be_renamed AS " + query); + assertUpdate("ALTER VIEW test_view_to_be_renamed RENAME TO test_view_renamed"); + assertQuery("SELECT * FROM test_view_renamed", query); + + assertUpdate("CREATE SCHEMA test_different_schema"); + assertQueryFails("ALTER VIEW test_view_renamed RENAME TO test_different_schema.test_view_renamed", "line 1:1: View rename across schemas is not supported"); + assertUpdate("DROP VIEW test_view_renamed"); + assertUpdate("DROP SCHEMA test_different_schema"); + } + + @Test + public void testRenameViewIfNotExists() + { + assertQueryFails("ALTER VIEW test_rename_view_not_exist RENAME TO test_renamed_view_not_exist", "line 1:1: View 'memory.default.test_rename_view_not_exist' does not exist"); + assertQuerySucceeds("ALTER VIEW IF EXISTS test_rename_view_not_exist RENAME TO test_renamed_view_not_exist"); + } + private List listMemoryTables() { return getQueryRunner().listTables(getSession(), "memory", "default"); From b2da48dc15356f1e125d3bde2ca7cbdd504e74ce Mon Sep 17 00:00:00 2001 From: auden-woolfson Date: Thu, 31 Oct 2024 10:37:33 -0700 Subject: [PATCH 13/25] Migrate query state info resource to re2j regex and add test --- .../presto/server/QueryStateInfoResource.java | 5 ++- .../server/TestQueryStateInfoResource.java | 44 ++++++++++++++++++- 2 files changed, 45 insertions(+), 4 deletions(-) diff --git a/presto-main/src/main/java/com/facebook/presto/server/QueryStateInfoResource.java b/presto-main/src/main/java/com/facebook/presto/server/QueryStateInfoResource.java index bee344552699..0682706fac73 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/QueryStateInfoResource.java +++ b/presto-main/src/main/java/com/facebook/presto/server/QueryStateInfoResource.java @@ -20,6 +20,8 @@ import com.facebook.presto.resourcemanager.ResourceManagerProxy; import com.facebook.presto.spi.QueryId; import com.facebook.presto.spi.resourceGroups.ResourceGroupId; +import com.google.re2j.Pattern; +import io.airlift.slice.Slices; import javax.annotation.security.RolesAllowed; import javax.inject.Inject; @@ -45,7 +47,6 @@ import java.util.NoSuchElementException; import java.util.Optional; import java.util.OptionalInt; -import java.util.regex.Pattern; import static com.facebook.presto.execution.QueryState.QUEUED; import static com.facebook.presto.server.QueryStateInfo.createQueryStateInfo; @@ -107,7 +108,7 @@ public void getQueryStateInfos( List queryStateInfos = queryInfos.stream() .filter(queryInfo -> includeAllQueries || !queryInfo.getState().isDone()) - .filter(queryInfo -> userPattern.map(pattern -> pattern.matcher(queryInfo.getSession().getUser()).matches()).orElse(true)) + .filter(queryInfo -> userPattern.map(pattern -> pattern.matcher(Slices.utf8Slice(queryInfo.getSession().getUser())).matches()).orElse(true)) .map(queryInfo -> getQueryStateInfo( queryInfo, includeAllQueryProgressStats, diff --git a/presto-main/src/test/java/com/facebook/presto/server/TestQueryStateInfoResource.java b/presto-main/src/test/java/com/facebook/presto/server/TestQueryStateInfoResource.java index a7093dd97128..dbb97ccdd3b2 100644 --- a/presto-main/src/test/java/com/facebook/presto/server/TestQueryStateInfoResource.java +++ b/presto-main/src/test/java/com/facebook/presto/server/TestQueryStateInfoResource.java @@ -40,6 +40,7 @@ import static com.facebook.presto.testing.assertions.Assert.assertEquals; import static java.nio.charset.StandardCharsets.UTF_8; import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertThrows; import static org.testng.Assert.assertTrue; @Test(singleThreaded = true) @@ -80,12 +81,20 @@ public void setup() QueryResults queryResults2 = client.execute(request2, createJsonResponseHandler(jsonCodec(QueryResults.class))); client.execute(prepareGet().setUri(queryResults2.getNextUri()).build(), createJsonResponseHandler(QUERY_RESULTS_JSON_CODEC)); + Request request3 = preparePost() + .setUri(uriBuilderFrom(server.getBaseUrl()).replacePath("/v1/statement").build()) + .setBodyGenerator(createStaticBodyGenerator(LONG_LASTING_QUERY, UTF_8)) + .setHeader(PRESTO_USER, "alt3") + .build(); + QueryResults queryResults3 = client.execute(request3, createJsonResponseHandler(jsonCodec(QueryResults.class))); + client.execute(prepareGet().setUri(queryResults3.getNextUri()).build(), createJsonResponseHandler(QUERY_RESULTS_JSON_CODEC)); + // queries are started in the background, so they may not all be immediately visible while (true) { List queryInfos = client.execute( prepareGet().setUri(uriBuilderFrom(server.getBaseUrl()).replacePath("/v1/query").build()).build(), createJsonResponseHandler(listJsonCodec(BasicQueryInfo.class))); - if ((queryInfos.size() == 2) && queryInfos.stream().allMatch(info -> info.getState() == RUNNING)) { + if ((queryInfos.size() == 3) && queryInfos.stream().allMatch(info -> info.getState() == RUNNING)) { break; } } @@ -105,7 +114,7 @@ public void testGetAllQueryStateInfos() prepareGet().setUri(server.resolve("/v1/queryState")).build(), createJsonResponseHandler(listJsonCodec(QueryStateInfo.class))); - assertEquals(infos.size(), 2); + assertEquals(infos.size(), 3); } @Test @@ -138,6 +147,37 @@ public void testGetQueryStateInfo() assertNotNull(info); } + @Test + public void testQueryStateInfoRegexMatching() + { + // Match strings containing "user" + List infos = client.execute( + prepareGet() + .setUri(server.resolve("/v1/queryState?user=user%5Cd")) // Encoded user\d + .build(), + createJsonResponseHandler(listJsonCodec(QueryStateInfo.class))); + assertEquals(infos.size(), 2); + + // Match strings ending in a digit + infos = client.execute( + prepareGet() + .setUri(server.resolve("/v1/queryState?user=%2E%2A%5Cd")) // Encoded .*\d + .build(), + createJsonResponseHandler(listJsonCodec(QueryStateInfo.class))); + assertEquals(infos.size(), 3); + + // Non supported version of previous pattern + infos = client.execute( + prepareGet() + .setUri(server.resolve("/v1/queryState?user=%2E%7B%7D%5Cd")) // Encoded .{}\d + .build(), + createJsonResponseHandler(listJsonCodec(QueryStateInfo.class))); + assertEquals(infos.size(), 0); + + Request failingReq = prepareGet().setUri(server.resolve("/v1/queryState?user=%2A%5Cd")).build(); // Encoded *\d + assertThrows(() -> client.execute(failingReq, createJsonResponseHandler(listJsonCodec(QueryStateInfo.class)))); + } + @Test(expectedExceptions = {UnexpectedResponseException.class}, expectedExceptionsMessageRegExp = "Expected response code .*, but was 404") public void testGetQueryStateInfoNo() { From 85cc36af2cca5be4b15d17884f3f6a01bd107781 Mon Sep 17 00:00:00 2001 From: wangd Date: Wed, 4 Dec 2024 11:24:24 +0800 Subject: [PATCH 14/25] Use `Optional.orElseGet` than `orElse` to avoid unnecessary calculation --- .../bytecode/expression/BytecodeExpressionAssertions.java | 2 +- .../java/com/facebook/presto/delta/DeltaMetadata.java | 3 ++- .../facebook/presto/delta/DeltaPageSourceProvider.java | 2 +- .../com/facebook/presto/google/sheets/SheetsMetadata.java | 2 +- .../hive/metastore/SemiTransactionalHiveMetastore.java | 4 ++-- .../main/java/com/facebook/presto/hive/HiveMetadata.java | 6 +++--- .../java/com/facebook/presto/hive/HivePageSource.java | 2 +- .../src/main/java/com/facebook/presto/hive/HiveUtil.java | 3 ++- .../java/com/facebook/presto/hive/HiveWriterFactory.java | 2 +- .../com/facebook/presto/hive/ManifestPartitionLoader.java | 2 +- .../com/facebook/presto/hive/StoragePartitionLoader.java | 2 +- .../presto/hive/orc/OrcBatchPageSourceFactory.java | 2 +- .../presto/hive/orc/OrcSelectivePageSourceFactory.java | 2 +- .../presto/hive/parquet/ParquetPageSourceFactory.java | 2 +- .../java/com/facebook/presto/hive/HiveQueryRunner.java | 2 +- .../java/com/facebook/presto/iceberg/IcebergPageSink.java | 2 +- .../presto/iceberg/IcebergPageSourceProvider.java | 2 +- .../com/facebook/presto/iceberg/IcebergSplitManager.java | 3 ++- .../presto/iceberg/delete/IcebergDeletePageSink.java | 2 +- .../iceberg/optimizer/IcebergEqualityDeleteAsJoin.java | 2 +- .../presto/iceberg/rest/IcebergRestCatalogFactory.java | 2 +- .../com/facebook/presto/iceberg/util/StatisticsUtil.java | 2 +- .../presto/iceberg/IcebergDistributedTestBase.java | 2 +- .../src/main/java/com/facebook/presto/Session.java | 3 ++- .../facebook/presto/cost/ComparisonStatsCalculator.java | 3 ++- .../facebook/presto/dispatcher/LocalDispatchQuery.java | 8 ++++---- .../com/facebook/presto/execution/SqlStageExecution.java | 2 +- .../execution/resourceGroups/StochasticPriorityQueue.java | 2 +- .../presto/metadata/PolymorphicScalarFunctionBuilder.java | 2 +- .../com/facebook/presto/operator/LookupJoinOperators.java | 8 ++++---- .../java/com/facebook/presto/operator/PagesIndex.java | 4 ++-- .../SqlInvokedScalarFromAnnotationsParser.java | 4 ++-- .../java/com/facebook/presto/server/TaskResource.java | 2 +- .../presto/server/protocol/QueryResourceUtil.java | 3 ++- .../analyzer/MaterializedViewInformationExtractor.java | 2 +- .../facebook/presto/sql/analyzer/StatementAnalyzer.java | 2 +- .../com/facebook/presto/sql/gen/ExpressionCompiler.java | 2 +- .../java/com/facebook/presto/sql/gen/JoinCompiler.java | 4 ++-- .../presto/sql/planner/LocalExecutionPlanner.java | 8 ++++---- .../com/facebook/presto/sql/planner/TranslationMap.java | 2 +- .../presto/sql/planner/iterative/PlanNodeMatcher.java | 2 +- .../sql/planner/iterative/rule/GatherAndMergeWindows.java | 2 +- .../planner/iterative/rule/ProjectOffPushDownRule.java | 2 +- .../iterative/rule/PruneAggregationSourceColumns.java | 6 +++--- .../planner/iterative/rule/PruneJoinChildrenColumns.java | 2 +- .../planner/iterative/rule/PruneMarkDistinctColumns.java | 2 +- .../sql/planner/iterative/rule/PruneOutputColumns.java | 2 +- .../sql/planner/iterative/rule/PruneSemiJoinColumns.java | 2 +- .../rule/PruneSemiJoinFilteringSourceColumns.java | 4 ++-- .../iterative/rule/RewriteCaseExpressionPredicate.java | 2 +- .../sql/planner/optimizations/PushdownSubfields.java | 2 +- .../com/facebook/presto/sql/relational/Expressions.java | 2 +- .../sql/relational/SqlToRowExpressionTranslator.java | 2 +- .../presto/transaction/InMemoryTransactionManager.java | 2 +- .../sql/planner/assertions/RowExpressionVerifier.java | 2 +- .../com/facebook/presto/parquet/ColumnReaderFactory.java | 6 +++--- .../presto/pinot/query/PinotQueryGeneratorContext.java | 2 +- .../java/com/facebook/presto/rcfile/RcFileWriter.java | 3 ++- .../execution/AbstractPrestoSparkQueryExecution.java | 2 +- .../spark/execution/PrestoSparkRowOutputOperator.java | 2 +- .../task/PrestoSparkNativeTaskExecutorFactory.java | 2 +- .../main/java/com/facebook/presto/tpch/TpchMetadata.java | 2 +- 62 files changed, 89 insertions(+), 82 deletions(-) diff --git a/presto-bytecode/src/test/java/com/facebook/presto/bytecode/expression/BytecodeExpressionAssertions.java b/presto-bytecode/src/test/java/com/facebook/presto/bytecode/expression/BytecodeExpressionAssertions.java index 978268600768..7ee9fd51f024 100644 --- a/presto-bytecode/src/test/java/com/facebook/presto/bytecode/expression/BytecodeExpressionAssertions.java +++ b/presto-bytecode/src/test/java/com/facebook/presto/bytecode/expression/BytecodeExpressionAssertions.java @@ -113,7 +113,7 @@ public static Object execute(Function nodeGenerator, Parame System.out.println(tree); } - ClassLoader classLoader = parentClassLoader.orElse(BytecodeExpressionAssertions.class.getClassLoader()); + ClassLoader classLoader = parentClassLoader.orElseGet(BytecodeExpressionAssertions.class::getClassLoader); return classGenerator(classLoader) .defineClass(classDefinition, Object.class) diff --git a/presto-delta/src/main/java/com/facebook/presto/delta/DeltaMetadata.java b/presto-delta/src/main/java/com/facebook/presto/delta/DeltaMetadata.java index bf72dc81494f..ff391115bec2 100644 --- a/presto-delta/src/main/java/com/facebook/presto/delta/DeltaMetadata.java +++ b/presto-delta/src/main/java/com/facebook/presto/delta/DeltaMetadata.java @@ -280,7 +280,8 @@ public ConnectorTableMetadata getTableMetadata(ConnectorSession session, Connect @Override public List listTables(ConnectorSession session, Optional schemaName) { - List schemaNames = schemaName.>map(ImmutableList::of).orElse(listSchemaNames(session)); + List schemaNames = schemaName.>map(ImmutableList::of) + .orElseGet(() -> listSchemaNames(session)); ImmutableList.Builder tableNames = ImmutableList.builder(); for (String schema : schemaNames) { for (String tableName : metastore.getAllTables(metastoreContext(session), schema).orElse(emptyList())) { diff --git a/presto-delta/src/main/java/com/facebook/presto/delta/DeltaPageSourceProvider.java b/presto-delta/src/main/java/com/facebook/presto/delta/DeltaPageSourceProvider.java index 8d218275d6c6..9420438b6fd9 100644 --- a/presto-delta/src/main/java/com/facebook/presto/delta/DeltaPageSourceProvider.java +++ b/presto-delta/src/main/java/com/facebook/presto/delta/DeltaPageSourceProvider.java @@ -253,7 +253,7 @@ private static ConnectorPageSource createParquetPageSource( .map(type -> new MessageType(fileSchema.getName(), type)) .reduce(MessageType::union); - MessageType requestedSchema = message.orElse(new MessageType(fileSchema.getName(), ImmutableList.of())); + MessageType requestedSchema = message.orElseGet(() -> new MessageType(fileSchema.getName(), ImmutableList.of())); ImmutableList.Builder footerBlocks = ImmutableList.builder(); for (BlockMetaData block : parquetMetadata.getBlocks()) { diff --git a/presto-google-sheets/src/main/java/com/facebook/presto/google/sheets/SheetsMetadata.java b/presto-google-sheets/src/main/java/com/facebook/presto/google/sheets/SheetsMetadata.java index 4d2efbff12df..69b73d4c6915 100644 --- a/presto-google-sheets/src/main/java/com/facebook/presto/google/sheets/SheetsMetadata.java +++ b/presto-google-sheets/src/main/java/com/facebook/presto/google/sheets/SheetsMetadata.java @@ -161,7 +161,7 @@ public Map> listTableColumns(ConnectorSess @Override public List listTables(ConnectorSession session, Optional schemaName) { - String schema = schemaName.orElse(getOnlyElement(SCHEMAS)); + String schema = schemaName.orElseGet(() -> getOnlyElement(SCHEMAS)); if (listSchemaNames().contains(schema)) { return sheetsClient.getTableNames().stream() diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/SemiTransactionalHiveMetastore.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/SemiTransactionalHiveMetastore.java index 3d73bdfcaa64..e6e17535d11b 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/SemiTransactionalHiveMetastore.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/SemiTransactionalHiveMetastore.java @@ -1179,7 +1179,7 @@ private ConnectorCommitHandle commitShared() hdfsContext.getSession().map(MetastoreUtil::isUserDefinedTypeEncodingEnabled).orElse(false), columnConverterProvider, hdfsContext.getSession().map(ConnectorSession::getWarningCollector).orElse(NOOP), - hdfsContext.getSession().map(ConnectorSession::getRuntimeStats).orElse(new RuntimeStats())); + hdfsContext.getSession().map(ConnectorSession::getRuntimeStats).orElseGet(RuntimeStats::new)); switch (action.getType()) { case DROP: committer.prepareDropTable(metastoreContext, schemaTableName); @@ -1213,7 +1213,7 @@ private ConnectorCommitHandle commitShared() hdfsContext.getSession().map(MetastoreUtil::isUserDefinedTypeEncodingEnabled).orElse(false), columnConverterProvider, hdfsContext.getSession().map(ConnectorSession::getWarningCollector).orElse(NOOP), - hdfsContext.getSession().map(ConnectorSession::getRuntimeStats).orElse(new RuntimeStats())); + hdfsContext.getSession().map(ConnectorSession::getRuntimeStats).orElseGet(RuntimeStats::new)); switch (action.getType()) { case DROP: committer.prepareDropPartition(metastoreContext, schemaTableName, partitionValues); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java index abad533ec81d..5c38af670a6b 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java @@ -1720,7 +1720,7 @@ public Optional finishCreateTable(ConnectorSession sess HiveBasicStatistics basicStatistics = partitionUpdates.stream() .map(PartitionUpdate::getStatistics) .reduce((first, second) -> reduce(first, second, ADD)) - .orElse(createZeroStatistics()); + .orElseGet(() -> createZeroStatistics()); tableStatistics = createPartitionStatistics(session, basicStatistics, columnTypes, getColumnStatistics(partitionComputedStatistics, ImmutableList.of()), timeZone); } else { @@ -3180,14 +3180,14 @@ public Set listRoleGrants(ConnectorSession session, PrestoPrincipal p public void grantRoles(ConnectorSession session, Set roles, Set grantees, boolean withAdminOption, Optional grantor) { MetastoreContext metastoreContext = getMetastoreContext(session); - metastore.grantRoles(metastoreContext, roles, grantees, withAdminOption, grantor.orElse(new PrestoPrincipal(USER, session.getUser()))); + metastore.grantRoles(metastoreContext, roles, grantees, withAdminOption, grantor.orElseGet(() -> new PrestoPrincipal(USER, session.getUser()))); } @Override public void revokeRoles(ConnectorSession session, Set roles, Set grantees, boolean adminOptionFor, Optional grantor) { MetastoreContext metastoreContext = getMetastoreContext(session); - metastore.revokeRoles(metastoreContext, roles, grantees, adminOptionFor, grantor.orElse(new PrestoPrincipal(USER, session.getUser()))); + metastore.revokeRoles(metastoreContext, roles, grantees, adminOptionFor, grantor.orElseGet(() -> new PrestoPrincipal(USER, session.getUser()))); } @Override diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSource.java b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSource.java index 4a1a178b2608..63c78ded9b53 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSource.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSource.java @@ -98,7 +98,7 @@ public HivePageSource( } else if (isRowIdColumnHandle(columnMapping.getHiveColumnHandle())) { // If there's no row ID partition component, then path + row numbers will be supplied for $row_id - byte[] component = rowIdPartitionComponent.orElse(new byte[0]); + byte[] component = rowIdPartitionComponent.orElseGet(() -> new byte[0]); String rowGroupId = Paths.get(path).getFileName().toString(); coercers[columnIndex] = new RowIDCoercer(component, rowGroupId); } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveUtil.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveUtil.java index 87bdda732e86..34e879fb4f4a 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveUtil.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveUtil.java @@ -1237,8 +1237,9 @@ private static TypeSignature translateHiveUnsupportedTypeSignatureForTemporaryTa TypeSignatureParameter typeSignatureParameter = parameters.get(i); checkArgument(typeSignatureParameter.isNamedTypeSignature(), "unexpected row type signature parameter: %s", typeSignatureParameter); NamedTypeSignature namedTypeSignature = typeSignatureParameter.getNamedTypeSignature(); + int parameterIdx = i; updatedParameters.add(TypeSignatureParameter.of(new NamedTypeSignature( - Optional.of(namedTypeSignature.getFieldName().orElse(new RowFieldName("_field_" + i, false))), + Optional.of(namedTypeSignature.getFieldName().orElseGet(() -> new RowFieldName("_field_" + parameterIdx, false))), translateHiveUnsupportedTypeSignatureForTemporaryTable(namedTypeSignature.getTypeSignature())))); } return new TypeSignature(StandardTypes.ROW, updatedParameters.build()); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriterFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriterFactory.java index f08fdd87ef61..4ac892cb2167 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriterFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriterFactory.java @@ -401,7 +401,7 @@ public HiveWriter createWriter(Page partitionColumns, int position, OptionalInt hiveFileWriter = sortingFileWriterFactory.get().createSortingFileWriter( path, hiveFileWriter, - bucketNumber.orElse(abs(path.hashCode() % 1024)), + bucketNumber.orElseGet(() -> abs(path.hashCode() % 1024)), writerParameters.getWriteInfo().getTempPath()); } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/ManifestPartitionLoader.java b/presto-hive/src/main/java/com/facebook/presto/hive/ManifestPartitionLoader.java index de0f86e0ec04..0417c741b869 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/ManifestPartitionLoader.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/ManifestPartitionLoader.java @@ -157,7 +157,7 @@ private InternalHiveSplitFactory createInternalHiveSplitFactory( String inputFormatName = storage.getStorageFormat().getInputFormat(); int partitionDataColumnCount = partition.getPartition() .map(p -> p.getColumns().size()) - .orElse(table.getDataColumns().size()); + .orElseGet(table.getDataColumns()::size); List partitionKeys = getPartitionKeys(table, partition.getPartition(), partitionName); Path path = new Path(getPartitionLocation(table, partition.getPartition())); Configuration configuration = hdfsEnvironment.getConfiguration(hdfsContext, path); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/StoragePartitionLoader.java b/presto-hive/src/main/java/com/facebook/presto/hive/StoragePartitionLoader.java index 676a68f2757e..08ab8a7b5de7 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/StoragePartitionLoader.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/StoragePartitionLoader.java @@ -261,7 +261,7 @@ public ListenableFuture loadPartition(HivePartitionMetadata partition, HiveSp String inputFormatName = storage.getStorageFormat().getInputFormat(); int partitionDataColumnCount = partition.getPartition() .map(p -> p.getColumns().size()) - .orElse(table.getDataColumns().size()); + .orElseGet(table.getDataColumns()::size); List partitionKeys = getPartitionKeys(table, partition.getPartition(), partitionName); String location = getPartitionLocation(table, partition.getPartition()); if (location.isEmpty()) { diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcBatchPageSourceFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcBatchPageSourceFactory.java index c5e2139e12b2..0e3aef9254f7 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcBatchPageSourceFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcBatchPageSourceFactory.java @@ -241,7 +241,7 @@ public static ConnectorPageSource createOrcPageSource( systemMemoryUsage, INITIAL_BATCH_SIZE); - byte[] partitionID = rowIDPartitionComponent.orElse(new byte[0]); + byte[] partitionID = rowIDPartitionComponent.orElseGet(() -> new byte[0]); String rowGroupID = path.getName(); // none of the columns are row numbers diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcSelectivePageSourceFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcSelectivePageSourceFactory.java index 9b2392a36efe..6cc1054b59a3 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcSelectivePageSourceFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcSelectivePageSourceFactory.java @@ -285,7 +285,7 @@ public static ConnectorPageSource createOrcPageSource( boolean supplyRowIDs = selectedColumns.stream().anyMatch(column -> HiveColumnHandle.isRowIdColumnHandle(column)); checkArgument(!supplyRowIDs || rowIDPartitionComponent.isPresent(), "rowIDPartitionComponent required when supplying row IDs"); - byte[] partitionID = rowIDPartitionComponent.orElse(new byte[0]); + byte[] partitionID = rowIDPartitionComponent.orElseGet(() -> new byte[0]); String rowGroupId = path.getName(); DataSize maxMergeDistance = getOrcMaxMergeDistance(session); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSourceFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSourceFactory.java index 1a544e05213a..1e9259fb59e8 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSourceFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSourceFactory.java @@ -214,7 +214,7 @@ public static ConnectorPageSource createParquetPageSource( .map(type -> new MessageType(fileSchema.getName(), type)) .reduce(MessageType::union); - MessageType requestedSchema = message.orElse(new MessageType(fileSchema.getName(), ImmutableList.of())); + MessageType requestedSchema = message.orElseGet(() -> new MessageType(fileSchema.getName(), ImmutableList.of())); ImmutableList.Builder footerBlocks = ImmutableList.builder(); diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/HiveQueryRunner.java b/presto-hive/src/test/java/com/facebook/presto/hive/HiveQueryRunner.java index 6b8462d23cde..8cf90df570cc 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/HiveQueryRunner.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/HiveQueryRunner.java @@ -218,7 +218,7 @@ public static DistributedQueryRunner createQueryRunner( queryRunner.createCatalog("tpchstandard", "tpch", tpchProperties); ExtendedHiveMetastore metastore; - metastore = externalMetastore.orElse(getFileHiveMetastore(queryRunner)); + metastore = externalMetastore.orElseGet(() -> getFileHiveMetastore(queryRunner)); queryRunner.installPlugin(new HivePlugin(HIVE_CATALOG, Optional.of(metastore))); diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergPageSink.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergPageSink.java index 2a59f0d9d872..d7977cbb008e 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergPageSink.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergPageSink.java @@ -315,7 +315,7 @@ private WriteContext createWriter(Optional partitionData) { String fileName = fileFormat.addExtension(randomUUID().toString()); Path outputPath = partitionData.map(partition -> new Path(locationProvider.newDataLocation(partitionSpec, partition, fileName))) - .orElse(new Path(locationProvider.newDataLocation(fileName))); + .orElseGet(() -> new Path(locationProvider.newDataLocation(fileName))); IcebergFileWriter writer = fileWriterFactory.createFileWriter( outputPath, diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergPageSourceProvider.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergPageSourceProvider.java index 8b62406d4d5d..2e7036c85170 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergPageSourceProvider.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergPageSourceProvider.java @@ -282,7 +282,7 @@ private static ConnectorPageSourceWithRowPositions createParquetPageSource( .map(type -> new MessageType(fileSchema.getName(), type)) .reduce(MessageType::union); - MessageType requestedSchema = messageType.orElse(new MessageType(fileSchema.getName(), ImmutableList.of())); + MessageType requestedSchema = messageType.orElseGet(() -> new MessageType(fileSchema.getName(), ImmutableList.of())); Map, RichColumnDescriptor> descriptorsByPath = getDescriptors(fileSchema, requestedSchema); TupleDomain parquetTupleDomain = getParquetTupleDomain(descriptorsByPath, effectivePredicate); Predicate parquetPredicate = buildPredicate(requestedSchema, parquetTupleDomain, descriptorsByPath); diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergSplitManager.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergSplitManager.java index e8baf9ea5f5d..d824accafcab 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergSplitManager.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergSplitManager.java @@ -90,7 +90,8 @@ public ConnectorSplitSource getSplits( if (table.getIcebergTableName().getTableType() == CHANGELOG) { // if the snapshot isn't specified, grab the oldest available version of the table long fromSnapshot = table.getIcebergTableName().getSnapshotId().orElseGet(() -> SnapshotUtil.oldestAncestor(icebergTable).snapshotId()); - long toSnapshot = table.getIcebergTableName().getChangelogEndSnapshot().orElse(icebergTable.currentSnapshot().snapshotId()); + long toSnapshot = table.getIcebergTableName().getChangelogEndSnapshot() + .orElseGet(icebergTable.currentSnapshot()::snapshotId); IncrementalChangelogScan scan = icebergTable.newIncrementalChangelogScan() .fromSnapshotExclusive(fromSnapshot) .toSnapshot(toSnapshot); diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/delete/IcebergDeletePageSink.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/delete/IcebergDeletePageSink.java index 02fd5364decd..0a9043684ba8 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/delete/IcebergDeletePageSink.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/delete/IcebergDeletePageSink.java @@ -104,7 +104,7 @@ public IcebergDeletePageSink( this.partitionData = partitionDataFromJson(partitionSpec, partitionDataAsJson); String fileName = fileFormat.addExtension(String.format("delete_file_%s", randomUUID().toString())); this.outputPath = partitionData.map(partition -> new Path(locationProvider.newDataLocation(partitionSpec, partition, fileName))) - .orElse(new Path(locationProvider.newDataLocation(fileName))); + .orElseGet(() -> new Path(locationProvider.newDataLocation(fileName))); this.positionDeleteWriter = new IcebergPositionDeleteWriter(); } diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/optimizer/IcebergEqualityDeleteAsJoin.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/optimizer/IcebergEqualityDeleteAsJoin.java index d5d8a919c6a3..f54d2e78b128 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/optimizer/IcebergEqualityDeleteAsJoin.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/optimizer/IcebergEqualityDeleteAsJoin.java @@ -182,7 +182,7 @@ public PlanNode visitTableScan(TableScanNode node, RewriteContext context) TupleDomain predicate = icebergTableLayoutHandle .map(IcebergTableLayoutHandle::getValidPredicate) .map(IcebergUtil::getNonMetadataColumnConstraints) - .orElse(TupleDomain.all()); + .orElseGet(TupleDomain::all); // Collect info about each unique delete schema to join by ImmutableMap, DeleteSetInfo> deleteSchemas = collectDeleteInformation(icebergTable, predicate, tableName.getSnapshotId().get()); diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/rest/IcebergRestCatalogFactory.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/rest/IcebergRestCatalogFactory.java index 3f4d9422d014..0c588d71e2ef 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/rest/IcebergRestCatalogFactory.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/rest/IcebergRestCatalogFactory.java @@ -169,7 +169,7 @@ protected SessionContext convertSession(ConnectorSession session) .setIdentity(session.getUser()) .setCredentials(credentials.build()) .setProperties(properties); - }).orElse(builder(session).setSessionId(randomUUID().toString())); + }).orElseGet(() -> builder(session).setSessionId(randomUUID().toString())); return sessionContextBuilder.build(); } diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/util/StatisticsUtil.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/util/StatisticsUtil.java index f19ae11b0795..cf28a7fdaeb0 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/util/StatisticsUtil.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/util/StatisticsUtil.java @@ -133,7 +133,7 @@ public static EnumSet decodeMergeFlags(String input) .collect(Collectors.toSet())) .filter(set -> !set.isEmpty()) .map(EnumSet::copyOf) - .orElse(EnumSet.noneOf(ColumnStatisticType.class)); + .orElseGet(() -> EnumSet.noneOf(ColumnStatisticType.class)); } public static String encodeMergeFlags(EnumSet flags) diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/IcebergDistributedTestBase.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/IcebergDistributedTestBase.java index ebc9814c774b..5bdaa3db84ae 100644 --- a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/IcebergDistributedTestBase.java +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/IcebergDistributedTestBase.java @@ -1064,7 +1064,7 @@ private TableStatistics getTableStats(String name, Optional snapshot, Sess handle, new ArrayList<>(columns .map(columnSet -> Maps.filterKeys(resolver.getColumnHandles(handle), columnSet::contains)) - .orElse(resolver.getColumnHandles(handle)).values()), + .orElseGet(() -> resolver.getColumnHandles(handle)).values()), Constraint.alwaysTrue()); } diff --git a/presto-main/src/main/java/com/facebook/presto/Session.java b/presto-main/src/main/java/com/facebook/presto/Session.java index 2e8c35f510d8..c67959db9176 100644 --- a/presto-main/src/main/java/com/facebook/presto/Session.java +++ b/presto-main/src/main/java/com/facebook/presto/Session.java @@ -845,7 +845,8 @@ public Session build() Optional.ofNullable(userAgent), Optional.ofNullable(clientInfo), clientTags, - Optional.ofNullable(resourceEstimates).orElse(new ResourceEstimateBuilder().build()), + Optional.ofNullable(resourceEstimates) + .orElseGet(new ResourceEstimateBuilder()::build), startTime, systemProperties, connectorProperties, diff --git a/presto-main/src/main/java/com/facebook/presto/cost/ComparisonStatsCalculator.java b/presto-main/src/main/java/com/facebook/presto/cost/ComparisonStatsCalculator.java index efc05bf4a20a..cba986d84bcf 100644 --- a/presto-main/src/main/java/com/facebook/presto/cost/ComparisonStatsCalculator.java +++ b/presto-main/src/main/java/com/facebook/presto/cost/ComparisonStatsCalculator.java @@ -174,7 +174,8 @@ private double calculateFilterFactor(VariableStatsEstimate variableStatistics, S Estimate filterEstimate; if (useHistograms) { Estimate distinctEstimate = isNaN(variableStatistics.getDistinctValuesCount()) ? Estimate.unknown() : Estimate.of(variableRange.getDistinctValuesCount()); - filterEstimate = HistogramCalculator.calculateFilterFactor(intersectRange.toPrestoRange(), intersectRange.getDistinctValuesCount(), variableStatistics.getHistogram().orElse(new UniformDistributionHistogram(variableStatistics.getLowValue(), variableStatistics.getHighValue())), distinctEstimate, true); + filterEstimate = HistogramCalculator.calculateFilterFactor(intersectRange.toPrestoRange(), intersectRange.getDistinctValuesCount(), + variableStatistics.getHistogram().orElseGet(() -> new UniformDistributionHistogram(variableStatistics.getLowValue(), variableStatistics.getHighValue())), distinctEstimate, true); if (log.isDebugEnabled()) { double expressionFilter = variableRange.overlapPercentWith(intersectRange); if (!Double.isNaN(expressionFilter) && diff --git a/presto-main/src/main/java/com/facebook/presto/dispatcher/LocalDispatchQuery.java b/presto-main/src/main/java/com/facebook/presto/dispatcher/LocalDispatchQuery.java index 50b7dea3fba3..0de6ee95f676 100644 --- a/presto-main/src/main/java/com/facebook/presto/dispatcher/LocalDispatchQuery.java +++ b/presto-main/src/main/java/com/facebook/presto/dispatcher/LocalDispatchQuery.java @@ -299,7 +299,7 @@ public Duration getTotalCpuTime() { return tryGetQueryExecution() .map(QueryExecution::getTotalCpuTime) - .orElse(new Duration(0, MILLISECONDS)); + .orElseGet(() -> new Duration(0, MILLISECONDS)); } @Override @@ -307,7 +307,7 @@ public DataSize getTotalMemoryReservation() { return tryGetQueryExecution() .map(QueryExecution::getTotalMemoryReservation) - .orElse(new DataSize(0, BYTE)); + .orElseGet(() -> new DataSize(0, BYTE)); } @Override @@ -315,7 +315,7 @@ public DataSize getUserMemoryReservation() { return tryGetQueryExecution() .map(QueryExecution::getUserMemoryReservation) - .orElse(new DataSize(0, BYTE)); + .orElseGet(() -> new DataSize(0, BYTE)); } public int getRunningTaskCount() @@ -328,7 +328,7 @@ public BasicQueryInfo getBasicQueryInfo() { return tryGetQueryExecution() .map(QueryExecution::getBasicQueryInfo) - .orElse(stateMachine.getBasicQueryInfo(Optional.empty())); + .orElseGet(() -> stateMachine.getBasicQueryInfo(Optional.empty())); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/execution/SqlStageExecution.java b/presto-main/src/main/java/com/facebook/presto/execution/SqlStageExecution.java index f3d03ead61cb..ee2159eba56c 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/SqlStageExecution.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/SqlStageExecution.java @@ -610,7 +610,7 @@ private void updateTaskStatus(TaskId taskId, TaskStatus taskStatus) .findFirst() .map(this::rewriteTransportFailure) .map(ExecutionFailureInfo::toException) - .orElse(new PrestoException(GENERIC_INTERNAL_ERROR, "A task failed for an unknown reason")); + .orElseGet(() -> new PrestoException(GENERIC_INTERNAL_ERROR, "A task failed for an unknown reason")); if (isRecoverable(taskStatus.getFailures())) { try { stageTaskRecoveryCallback.get().recover(taskId); diff --git a/presto-main/src/main/java/com/facebook/presto/execution/resourceGroups/StochasticPriorityQueue.java b/presto-main/src/main/java/com/facebook/presto/execution/resourceGroups/StochasticPriorityQueue.java index 709190db42e4..5f51526402ef 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/resourceGroups/StochasticPriorityQueue.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/resourceGroups/StochasticPriorityQueue.java @@ -219,7 +219,7 @@ public Node findLeaf() int rightDescendants = right.map(node -> node.descendants).orElse(0); if (leftDescendants == 0 && rightDescendants == 0) { - return left.orElse(right.orElse(this)); + return left.orElseGet(() -> right.orElse(this)); } if (leftDescendants > rightDescendants) { return left.get().findLeaf(); diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/PolymorphicScalarFunctionBuilder.java b/presto-main/src/main/java/com/facebook/presto/metadata/PolymorphicScalarFunctionBuilder.java index 805f0eee62a7..612913e5c4d9 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/PolymorphicScalarFunctionBuilder.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/PolymorphicScalarFunctionBuilder.java @@ -67,7 +67,7 @@ public PolymorphicScalarFunctionBuilder(Class clazz, OperatorType operatorTyp public PolymorphicScalarFunctionBuilder signature(Signature signature) { this.signature = requireNonNull(signature, "signature is null"); - this.visibility = Optional.of(visibility.orElse(isOperator(signature) ? HIDDEN : PUBLIC)); + this.visibility = Optional.of(visibility.orElseGet(() -> isOperator(signature) ? HIDDEN : PUBLIC)); return this; } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/LookupJoinOperators.java b/presto-main/src/main/java/com/facebook/presto/operator/LookupJoinOperators.java index 332573af9475..e3918b42be0f 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/LookupJoinOperators.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/LookupJoinOperators.java @@ -77,7 +77,7 @@ public OperatorFactory innerJoin( probeTypes, probeJoinChannel, probeHashChannel, - probeOutputChannels.orElse(rangeList(probeTypes.size())), + probeOutputChannels.orElseGet(() -> rangeList(probeTypes.size())), JoinType.INNER, totalOperatorsCount, partitioningSpillerFactory, @@ -103,7 +103,7 @@ public OperatorFactory probeOuterJoin( probeTypes, probeJoinChannel, probeHashChannel, - probeOutputChannels.orElse(rangeList(probeTypes.size())), + probeOutputChannels.orElseGet(() -> rangeList(probeTypes.size())), JoinType.PROBE_OUTER, totalOperatorsCount, partitioningSpillerFactory, @@ -129,7 +129,7 @@ public OperatorFactory lookupOuterJoin( probeTypes, probeJoinChannel, probeHashChannel, - probeOutputChannels.orElse(rangeList(probeTypes.size())), + probeOutputChannels.orElseGet(() -> rangeList(probeTypes.size())), JoinType.LOOKUP_OUTER, totalOperatorsCount, partitioningSpillerFactory, @@ -155,7 +155,7 @@ public OperatorFactory fullOuterJoin( probeTypes, probeJoinChannel, probeHashChannel, - probeOutputChannels.orElse(rangeList(probeTypes.size())), + probeOutputChannels.orElseGet(() -> rangeList(probeTypes.size())), JoinType.FULL_OUTER, totalOperatorsCount, partitioningSpillerFactory, diff --git a/presto-main/src/main/java/com/facebook/presto/operator/PagesIndex.java b/presto-main/src/main/java/com/facebook/presto/operator/PagesIndex.java index 1b3e3fbb6137..bd2f5ffbc817 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/PagesIndex.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/PagesIndex.java @@ -429,7 +429,7 @@ public PagesHashStrategy createPagesHashStrategy(List joinChannels, Opt // if compilation fails, use interpreter return new SimplePagesHashStrategy( types, - outputChannels.orElse(rangeList(types.size())), + outputChannels.orElseGet(() -> rangeList(types.size())), ImmutableList.copyOf(channels), joinChannels, hashChannel, @@ -505,7 +505,7 @@ public LookupSourceSupplier createLookupSourceSupplier( // if compilation fails PagesHashStrategy hashStrategy = new SimplePagesHashStrategy( types, - outputChannels.orElse(rangeList(types.size())), + outputChannels.orElseGet(() -> rangeList(types.size())), channels, joinChannels, hashChannel, diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/annotations/SqlInvokedScalarFromAnnotationsParser.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/annotations/SqlInvokedScalarFromAnnotationsParser.java index fa6eeb8d8e6f..2517bc19b908 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/annotations/SqlInvokedScalarFromAnnotationsParser.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/annotations/SqlInvokedScalarFromAnnotationsParser.java @@ -122,8 +122,8 @@ private static List findScalarsInFunctionSetClass(Class clazz) private static List createSqlInvokedFunctions(Method method, Optional header, Optional description) { - SqlInvokedScalarFunction functionHeader = header.orElse(method.getAnnotation(SqlInvokedScalarFunction.class)); - String functionDescription = description.orElse(method.isAnnotationPresent(Description.class) ? method.getAnnotation(Description.class).value() : ""); + SqlInvokedScalarFunction functionHeader = header.orElseGet(() -> method.getAnnotation(SqlInvokedScalarFunction.class)); + String functionDescription = description.orElseGet(() -> method.isAnnotationPresent(Description.class) ? method.getAnnotation(Description.class).value() : ""); TypeSignature returnType = parseTypeSignature(method.getAnnotation(SqlType.class).value()); // Parameter diff --git a/presto-main/src/main/java/com/facebook/presto/server/TaskResource.java b/presto-main/src/main/java/com/facebook/presto/server/TaskResource.java index 372326ad4ce4..17dc3450790e 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/TaskResource.java +++ b/presto-main/src/main/java/com/facebook/presto/server/TaskResource.java @@ -314,7 +314,7 @@ public Response taskResultsHeaders( .build(); }) .findFirst() - .orElse(Response.status(Response.Status.NOT_FOUND).build()); + .orElseGet(Response.status(Response.Status.NOT_FOUND)::build); } @HEAD diff --git a/presto-main/src/main/java/com/facebook/presto/server/protocol/QueryResourceUtil.java b/presto-main/src/main/java/com/facebook/presto/server/protocol/QueryResourceUtil.java index 79cdc1ba335c..b713d7b60cac 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/protocol/QueryResourceUtil.java +++ b/presto-main/src/main/java/com/facebook/presto/server/protocol/QueryResourceUtil.java @@ -424,7 +424,8 @@ public static QueryResults createQueuedQueryResults( Optional queuedTime, Duration waitingForPrerequisitesTime) { - QueryState state = queryError.map(error -> FAILED).orElse(queuedTime.isPresent() ? QUEUED : WAITING_FOR_PREREQUISITES); + QueryState state = queryError.map(error -> FAILED) + .orElseGet(() -> queuedTime.isPresent() ? QUEUED : WAITING_FOR_PREREQUISITES); return new QueryResults( queryId.toString(), getQueryHtmlUri(queryId, uriInfo, xForwardedProto, xPrestoPrefixUrl), diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/MaterializedViewInformationExtractor.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/MaterializedViewInformationExtractor.java index 432ce0a3f2ec..0c6cf0402a41 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/MaterializedViewInformationExtractor.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/MaterializedViewInformationExtractor.java @@ -111,7 +111,7 @@ private void addBaseToViewColumn(SingleColumn singleColumn) if (key instanceof FunctionCall && !singleColumn.getAlias().isPresent()) { throw new SemanticException(NOT_SUPPORTED, singleColumn, "Derived field in materialized view must have an alias"); } - baseToViewColumnMap.put(key, singleColumn.getAlias().orElse(new Identifier(key.toString()))); + baseToViewColumnMap.put(key, singleColumn.getAlias().orElseGet(() -> new Identifier(key.toString()))); } private void addGroupBy(GroupingElement groupingElement) diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/StatementAnalyzer.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/StatementAnalyzer.java index 1e4e86af0644..d52540a1660f 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/StatementAnalyzer.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/StatementAnalyzer.java @@ -2355,7 +2355,7 @@ private List analyzeWindowFunctions(QuerySpecification node, List< private void analyzeWindowFrame(WindowFrame frame) { FrameBound.Type startType = frame.getStart().getType(); - FrameBound.Type endType = frame.getEnd().orElse(new FrameBound(CURRENT_ROW)).getType(); + FrameBound.Type endType = frame.getEnd().orElseGet(() -> new FrameBound(CURRENT_ROW)).getType(); if (startType == UNBOUNDED_FOLLOWING) { throw new SemanticException(INVALID_WINDOW_FRAME, frame, "Window frame start cannot be UNBOUNDED FOLLOWING"); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/gen/ExpressionCompiler.java b/presto-main/src/main/java/com/facebook/presto/sql/gen/ExpressionCompiler.java index 93b69f577d5e..4721e326daf8 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/gen/ExpressionCompiler.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/gen/ExpressionCompiler.java @@ -207,7 +207,7 @@ private Class compile( { // create filter and project page iterator class try { - return compileProcessor(sqlFunctionProperties, filter.orElse(constant(true, BOOLEAN)), projections, bodyCompiler, superType); + return compileProcessor(sqlFunctionProperties, filter.orElseGet(() -> constant(true, BOOLEAN)), projections, bodyCompiler, superType); } catch (CompilationException e) { throw new PrestoException(COMPILER_ERROR, e.getCause()); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/gen/JoinCompiler.java b/presto-main/src/main/java/com/facebook/presto/sql/gen/JoinCompiler.java index 07f33df58ede..207a7fa9640f 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/gen/JoinCompiler.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/gen/JoinCompiler.java @@ -134,7 +134,7 @@ public LookupSourceSupplierFactory compileLookupSourceFactory(List rangeList(types.size())), joinChannels, sortChannel)); } @@ -152,7 +152,7 @@ public PagesHashStrategyFactory compilePagesHashStrategyFactory(List types return new PagesHashStrategyFactory(hashStrategies.getUnchecked(new CacheKey( types, - outputChannels.orElse(rangeList(types.size())), + outputChannels.orElseGet(() -> rangeList(types.size())), joinChannels, Optional.empty()))); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/LocalExecutionPlanner.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/LocalExecutionPlanner.java index c1b80bdf1ad3..fbde1ea09421 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/LocalExecutionPlanner.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/LocalExecutionPlanner.java @@ -2735,7 +2735,7 @@ public PhysicalOperation visitTableWriter(TableWriterNode node, LocalExecutionPl // turns PARTIAL aggregation into INTERMEDIATE. Optional.empty(), true); - }).orElse(new DevNullOperatorFactory(context.getNextOperatorId(), node.getId())); + }).orElseGet(() -> new DevNullOperatorFactory(context.getNextOperatorId(), node.getId())); List inputChannels = node.getColumns().stream() .map(source::variableToChannel) @@ -2836,7 +2836,7 @@ public PhysicalOperation visitTableWriteMerge(TableWriterMergeNode node, LocalEx 200, Optional.empty(), true); - }).orElse(new DevNullOperatorFactory(context.getNextOperatorId(), node.getId())); + }).orElseGet(() -> new DevNullOperatorFactory(context.getNextOperatorId(), node.getId())); OperatorFactory operatorFactory = new TableWriterMergeOperatorFactory( context.getNextOperatorId(), @@ -2892,12 +2892,12 @@ public PhysicalOperation visitTableFinish(TableFinishNode node, LocalExecutionPl // final aggregation ignores partial pre-aggregation memory limit Optional.empty(), true); - }).orElse(new DevNullOperatorFactory(context.getNextOperatorId(), node.getId())); + }).orElseGet(() -> new DevNullOperatorFactory(context.getNextOperatorId(), node.getId())); Map aggregationOutput = outputMapping.build(); StatisticAggregationsDescriptor descriptor = node.getStatisticsAggregationDescriptor() .map(desc -> desc.map(aggregationOutput::get)) - .orElse(StatisticAggregationsDescriptor.empty()); + .orElseGet(StatisticAggregationsDescriptor::empty); ExecutionWriterTarget writerTarget = context.getTableWriteInfo().getWriterTarget().orElseThrow(() -> new VerifyException("writerTarget is absent")); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/TranslationMap.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/TranslationMap.java index e06cf4a67e31..6b23646a8916 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/TranslationMap.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/TranslationMap.java @@ -230,7 +230,7 @@ private Expression rewriteExpressionWithResolvedName(Expression node) { return getVariable(rewriteBase, node) .map(variable -> coerceIfNecessary(node, createSymbolReference(variable))) - .orElse(coerceIfNecessary(node, node)); + .orElseGet(() -> coerceIfNecessary(node, node)); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/PlanNodeMatcher.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/PlanNodeMatcher.java index 28756b7dee96..ba22a1f9b2b7 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/PlanNodeMatcher.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/PlanNodeMatcher.java @@ -42,7 +42,7 @@ public Match matchWith(WithPattern withPattern, Object object, Capture Match propertyMatch = resolvedValue .map(value -> match(withPattern.getPattern(), value, captures)) - .orElse(Match.empty()); + .orElseGet(Match::empty); return propertyMatch.map(ignored -> (T) object); } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/GatherAndMergeWindows.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/GatherAndMergeWindows.java index 1163a6ef3dd5..798a876d5947 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/GatherAndMergeWindows.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/GatherAndMergeWindows.java @@ -108,7 +108,7 @@ public Result apply(WindowNode parent, Captures captures, Context context) return pullWindowNodeAboveProjects(captures.get(childCapture), projects) .flatMap(newChild -> manipulateAdjacentWindowNodes(parent, newChild, context)) .map(Result::ofPlanNode) - .orElse(Result.empty()); + .orElseGet(Result::empty); } protected abstract Optional manipulateAdjacentWindowNodes(WindowNode parent, WindowNode child, Context context); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/ProjectOffPushDownRule.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/ProjectOffPushDownRule.java index baa87e947e97..b5e1da84d239 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/ProjectOffPushDownRule.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/ProjectOffPushDownRule.java @@ -70,7 +70,7 @@ public Result apply(ProjectNode parent, Captures captures, Context context) .flatMap(prunedOutputs -> this.pushDownProjectOff(context.getIdAllocator(), context.getVariableAllocator(), targetNode, prunedOutputs)) .map(newChild -> parent.replaceChildren(ImmutableList.of(newChild))) .map(Result::ofPlanNode) - .orElse(Result.empty()); + .orElseGet(Result::empty); } protected abstract Optional pushDownProjectOff(PlanNodeIdAllocator idAllocator, VariableAllocator variableAllocator, N targetNode, Set referencedOutputs); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneAggregationSourceColumns.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneAggregationSourceColumns.java index e10172bcf9e2..7f94f5de9304 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneAggregationSourceColumns.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneAggregationSourceColumns.java @@ -47,20 +47,20 @@ public Result apply(AggregationNode aggregationNode, Captures captures, Context { Set requiredInputs = Streams.concat( aggregationNode.getGroupingKeys().stream(), - aggregationNode.getHashVariable().map(Stream::of).orElse(Stream.empty()), + aggregationNode.getHashVariable().map(Stream::of).orElseGet(Stream::empty), aggregationNode.getAggregations().values().stream() .flatMap(aggregation -> getAggregationInputs(aggregation, TypeProvider.viewOf(context.getVariableAllocator().getVariables())))) .collect(toImmutableSet()); return restrictChildOutputs(context.getIdAllocator(), aggregationNode, requiredInputs) .map(Result::ofPlanNode) - .orElse(Result.empty()); + .orElseGet(Result::empty); } private static Stream getAggregationInputs(AggregationNode.Aggregation aggregation, TypeProvider types) { return Streams.concat( AggregationNodeUtils.extractAggregationUniqueVariables(aggregation).stream(), - aggregation.getMask().map(Stream::of).orElse(Stream.empty())); + aggregation.getMask().map(Stream::of).orElseGet(Stream::empty)); } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneJoinChildrenColumns.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneJoinChildrenColumns.java index d869f38bea8e..c7d5bd21e2b1 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneJoinChildrenColumns.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneJoinChildrenColumns.java @@ -78,6 +78,6 @@ public Result apply(JoinNode joinNode, Captures captures, Context context) return restrictChildOutputs(context.getIdAllocator(), joinNode, leftUsableInputs, rightUsableInputs) .map(Result::ofPlanNode) - .orElse(Result.empty()); + .orElseGet(Result::empty); } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneMarkDistinctColumns.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneMarkDistinctColumns.java index f229e774c946..8a2f18468158 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneMarkDistinctColumns.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneMarkDistinctColumns.java @@ -47,7 +47,7 @@ protected Optional pushDownProjectOff(PlanNodeIdAllocator idAllocator, referencedOutputs.stream() .filter(variable -> !variable.equals(markDistinctNode.getMarkerVariable())), markDistinctNode.getDistinctVariables().stream(), - markDistinctNode.getHashVariable().map(Stream::of).orElse(Stream.empty())) + markDistinctNode.getHashVariable().map(Stream::of).orElseGet(Stream::empty)) .collect(toImmutableSet()); return restrictChildOutputs(idAllocator, markDistinctNode, requiredInputs); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneOutputColumns.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneOutputColumns.java index dbb2aa652f92..9a6328b50f1e 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneOutputColumns.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneOutputColumns.java @@ -45,6 +45,6 @@ public Result apply(OutputNode outputNode, Captures captures, Context context) outputNode, ImmutableSet.copyOf(outputNode.getOutputVariables())) .map(Result::ofPlanNode) - .orElse(Result.empty()); + .orElseGet(Result::empty); } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneSemiJoinColumns.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneSemiJoinColumns.java index 6b32a40579e1..99c53839f151 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneSemiJoinColumns.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneSemiJoinColumns.java @@ -48,7 +48,7 @@ protected Optional pushDownProjectOff(PlanNodeIdAllocator idAllocator, referencedOutputs.stream() .filter(variable -> !variable.equals(semiJoinNode.getSemiJoinOutput())), Stream.of(semiJoinNode.getSourceJoinVariable()), - semiJoinNode.getSourceHashVariable().map(Stream::of).orElse(Stream.empty())) + semiJoinNode.getSourceHashVariable().map(Stream::of).orElseGet(Stream::empty)) .collect(toImmutableSet()); return restrictOutputs(idAllocator, semiJoinNode.getSource(), requiredSourceInputs) diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneSemiJoinFilteringSourceColumns.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneSemiJoinFilteringSourceColumns.java index eb09aa06359f..0efb07e67fab 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneSemiJoinFilteringSourceColumns.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneSemiJoinFilteringSourceColumns.java @@ -48,13 +48,13 @@ public Result apply(SemiJoinNode semiJoinNode, Captures captures, Context contex { Set requiredFilteringSourceInputs = Streams.concat( Stream.of(semiJoinNode.getFilteringSourceJoinVariable()), - semiJoinNode.getFilteringSourceHashVariable().map(Stream::of).orElse(Stream.empty())) + semiJoinNode.getFilteringSourceHashVariable().map(Stream::of).orElseGet(Stream::empty)) .collect(toImmutableSet()); return restrictOutputs(context.getIdAllocator(), semiJoinNode.getFilteringSource(), requiredFilteringSourceInputs) .map(newFilteringSource -> semiJoinNode.replaceChildren(ImmutableList.of(semiJoinNode.getSource(), newFilteringSource))) .map(Result::ofPlanNode) - .orElse(Result.empty()); + .orElseGet(Result::empty); } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RewriteCaseExpressionPredicate.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RewriteCaseExpressionPredicate.java index d6173b92eeb4..25402c62627d 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RewriteCaseExpressionPredicate.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RewriteCaseExpressionPredicate.java @@ -250,7 +250,7 @@ private RowExpression getElseExpression(Optional castExpression, { return elseValue.map( elseVal -> comparisonExpressionGenerator.apply(castExpression.map(castExp -> replaceArguments(castExp, elseVal)).orElse(elseVal) - )).orElse(new SpecialFormExpression(IS_NULL, BOOLEAN, value)); + )).orElseGet(() -> new SpecialFormExpression(IS_NULL, BOOLEAN, value)); } private Optional getCaseOperand(RowExpression expression) diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PushdownSubfields.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PushdownSubfields.java index 31f15d9a694e..ada7285410e2 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PushdownSubfields.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PushdownSubfields.java @@ -711,7 +711,7 @@ public Void visitCall(CallExpression call, Context context) } Set argumentIndicesContainingMapOrArray = functionDescriptor.getArgumentIndicesContainingMapOrArray() - .orElse(IntStream.range(0, call.getArguments().size()) + .orElseGet(() -> IntStream.range(0, call.getArguments().size()) .filter(argIndex -> isMapOrArrayOfRowType(call.getArguments().get(argIndex))) .boxed() .collect(toImmutableSet())); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/relational/Expressions.java b/presto-main/src/main/java/com/facebook/presto/sql/relational/Expressions.java index 8596b9c5d35d..373ce2bfd65a 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/relational/Expressions.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/relational/Expressions.java @@ -200,7 +200,7 @@ public static RowExpression buildSwitch(RowExpression operand, List constantNull(operand.getSourceLocation(), returnType))); return specialForm(SWITCH, returnType, arguments.build()); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/relational/SqlToRowExpressionTranslator.java b/presto-main/src/main/java/com/facebook/presto/sql/relational/SqlToRowExpressionTranslator.java index 236174134a46..a39180396d2a 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/relational/SqlToRowExpressionTranslator.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/relational/SqlToRowExpressionTranslator.java @@ -673,7 +673,7 @@ private RowExpression buildSwitch(RowExpression operand, List whenCl arguments.add(defaultValue .map((value) -> process(value, context)) - .orElse(constantNull(operand.getSourceLocation(), returnType))); + .orElseGet(() -> constantNull(operand.getSourceLocation(), returnType))); return specialForm(SWITCH, returnType, arguments.build()); } diff --git a/presto-main/src/main/java/com/facebook/presto/transaction/InMemoryTransactionManager.java b/presto-main/src/main/java/com/facebook/presto/transaction/InMemoryTransactionManager.java index 9551b310fda2..6b75493cedb8 100644 --- a/presto-main/src/main/java/com/facebook/presto/transaction/InMemoryTransactionManager.java +++ b/presto-main/src/main/java/com/facebook/presto/transaction/InMemoryTransactionManager.java @@ -665,7 +665,7 @@ public TransactionInfo getTransactionInfo() { Duration idleTime = Optional.ofNullable(idleStartTime.get()) .map(Duration::nanosSince) - .orElse(new Duration(0, MILLISECONDS)); + .orElseGet(() -> new Duration(0, MILLISECONDS)); // dereferencing this field is safe because the field is atomic @SuppressWarnings("FieldAccessNotGuarded") Optional writtenConnectorId = Optional.ofNullable(this.writtenConnectorId.get()); diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/RowExpressionVerifier.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/RowExpressionVerifier.java index 66e594f132b4..f950477c06dd 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/RowExpressionVerifier.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/RowExpressionVerifier.java @@ -210,7 +210,7 @@ protected Boolean visitIfExpression(IfExpression expected, RowExpression actual) } return process(expected.getCondition(), ((SpecialFormExpression) actual).getArguments().get(0)) && process(expected.getTrueValue(), ((SpecialFormExpression) actual).getArguments().get(1)) && - process(expected.getFalseValue().orElse(new NullLiteral()), ((SpecialFormExpression) actual).getArguments().get(2)); + process(expected.getFalseValue().orElseGet(() -> new NullLiteral()), ((SpecialFormExpression) actual).getArguments().get(2)); } @Override diff --git a/presto-parquet/src/main/java/com/facebook/presto/parquet/ColumnReaderFactory.java b/presto-parquet/src/main/java/com/facebook/presto/parquet/ColumnReaderFactory.java index 2520bf241ddc..485935d579fc 100644 --- a/presto-parquet/src/main/java/com/facebook/presto/parquet/ColumnReaderFactory.java +++ b/presto-parquet/src/main/java/com/facebook/presto/parquet/ColumnReaderFactory.java @@ -115,7 +115,7 @@ public static ColumnReader createReader(RichColumnDescriptor descriptor, boolean case BOOLEAN: return new BooleanColumnReader(descriptor); case INT32: - return createDecimalColumnReader(descriptor).orElse(new IntColumnReader(descriptor)); + return createDecimalColumnReader(descriptor).orElseGet(() -> new IntColumnReader(descriptor)); case INT64: if (isTimeStampMicrosType(descriptor)) { return new LongTimestampMicrosColumnReader(descriptor); @@ -123,7 +123,7 @@ public static ColumnReader createReader(RichColumnDescriptor descriptor, boolean if (isTimeMicrosType(descriptor)) { return new LongTimeMicrosColumnReader(descriptor); } - return createDecimalColumnReader(descriptor).orElse(new LongColumnReader(descriptor)); + return createDecimalColumnReader(descriptor).orElseGet(() -> new LongColumnReader(descriptor)); case INT96: return new TimestampColumnReader(descriptor); case FLOAT: @@ -131,7 +131,7 @@ public static ColumnReader createReader(RichColumnDescriptor descriptor, boolean case DOUBLE: return new DoubleColumnReader(descriptor); case BINARY: - return createDecimalColumnReader(descriptor).orElse(new BinaryColumnReader(descriptor)); + return createDecimalColumnReader(descriptor).orElseGet(() -> new BinaryColumnReader(descriptor)); case FIXED_LEN_BYTE_ARRAY: if (isUuidType(descriptor)) { return new BinaryColumnReader(descriptor); diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/query/PinotQueryGeneratorContext.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/query/PinotQueryGeneratorContext.java index 53516b848d96..e4c0a1efd1d9 100644 --- a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/query/PinotQueryGeneratorContext.java +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/query/PinotQueryGeneratorContext.java @@ -370,7 +370,7 @@ public PinotQueryGenerator.GeneratedPinotQuery toSqlQuery(PinotConfig pinotConfi throw new PinotException(PINOT_QUERY_GENERATOR_FAILURE, Optional.empty(), "Broker non aggregate queries have to have a limit"); } else { - queryLimit = limit.orElse(PinotSessionProperties.getLimitLargerForSegment(session)); + queryLimit = limit.orElseGet(() -> PinotSessionProperties.getLimitLargerForSegment(session)); } } else if (hasGroupBy()) { diff --git a/presto-rcfile/src/main/java/com/facebook/presto/rcfile/RcFileWriter.java b/presto-rcfile/src/main/java/com/facebook/presto/rcfile/RcFileWriter.java index ab9bf93777df..f31417618fba 100644 --- a/presto-rcfile/src/main/java/com/facebook/presto/rcfile/RcFileWriter.java +++ b/presto-rcfile/src/main/java/com/facebook/presto/rcfile/RcFileWriter.java @@ -169,7 +169,8 @@ public RcFileWriter( recordValidation(validation -> validation.setSyncSecond(syncSecond)); // initialize columns - RcFileCompressor compressor = codecName.map(codecFactory::createCompressor).orElse(new NoneCompressor()); + RcFileCompressor compressor = codecName.map(codecFactory::createCompressor) + .orElseGet(NoneCompressor::new); keySectionOutput = compressor.createCompressedSliceOutput((int) MIN_BUFFER_SIZE.toBytes(), (int) MAX_BUFFER_SIZE.toBytes()); keySectionOutput.close(); // output is recycled on first use which requires output to be closed columnEncoders = new ColumnEncoder[types.size()]; diff --git a/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/AbstractPrestoSparkQueryExecution.java b/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/AbstractPrestoSparkQueryExecution.java index 8423a7065a38..46d422c62b46 100644 --- a/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/AbstractPrestoSparkQueryExecution.java +++ b/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/AbstractPrestoSparkQueryExecution.java @@ -817,7 +817,7 @@ protected synchronized RddAndMore createRdd } } - Class outputType = outputTypeOptional.orElse(getOutputType(subPlan)); + Class outputType = outputTypeOptional.orElseGet(() -> getOutputType(subPlan)); JavaPairRDD rdd = rddFactory.createSparkRdd( sparkContext, session, diff --git a/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/PrestoSparkRowOutputOperator.java b/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/PrestoSparkRowOutputOperator.java index 8378ab760944..f6d946eb8599 100644 --- a/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/PrestoSparkRowOutputOperator.java +++ b/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/PrestoSparkRowOutputOperator.java @@ -81,7 +81,7 @@ public OperatorFactory createOutputOperator( Optional outputPartitioning, PagesSerdeFactory serdeFactory) { - OutputPartitioning partitioning = outputPartitioning.orElse(preDeterminedPartition.orElse(SINGLE_PARTITION)); + OutputPartitioning partitioning = outputPartitioning.orElseGet(() -> preDeterminedPartition.orElse(SINGLE_PARTITION)); return new PrestoSparkRowOutputOperatorFactory( operatorId, planNodeId, diff --git a/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/task/PrestoSparkNativeTaskExecutorFactory.java b/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/task/PrestoSparkNativeTaskExecutorFactory.java index 1822a1b0f7d7..48acd0bbce25 100644 --- a/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/task/PrestoSparkNativeTaskExecutorFactory.java +++ b/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/task/PrestoSparkNativeTaskExecutorFactory.java @@ -398,7 +398,7 @@ private static void processTaskInfoForErrorsOrCompletion(TaskInfo taskInfo) RuntimeException failure = taskInfo.getTaskStatus().getFailures().stream() .findFirst() .map(ExecutionFailureInfo::toException) - .orElse(new PrestoException(GENERIC_INTERNAL_ERROR, "Native task failed for an unknown reason")); + .orElseGet(() -> new PrestoException(GENERIC_INTERNAL_ERROR, "Native task failed for an unknown reason")); throw failure; } diff --git a/presto-tpch/src/main/java/com/facebook/presto/tpch/TpchMetadata.java b/presto-tpch/src/main/java/com/facebook/presto/tpch/TpchMetadata.java index 59ea08df610f..92ddc6eafab1 100644 --- a/presto-tpch/src/main/java/com/facebook/presto/tpch/TpchMetadata.java +++ b/presto-tpch/src/main/java/com/facebook/presto/tpch/TpchMetadata.java @@ -489,7 +489,7 @@ private TupleDomain toTupleDomain(Map Domain.singleValue(type, nullableValue.getValue())) .reduce((Domain::union)) - .orElse(Domain.none(type)); + .orElseGet(() -> Domain.none(type)); }))); } From 70188806a9a47d45cbbf6ef727962991c5b4054a Mon Sep 17 00:00:00 2001 From: Yihong Wang Date: Tue, 17 Dec 2024 14:01:34 -0800 Subject: [PATCH 15/25] fix CVE-2024-55565 update nanoid pkg to 3.3.8 to fix the CVE Signed-off-by: Yihong Wang --- presto-ui/src/yarn.lock | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/presto-ui/src/yarn.lock b/presto-ui/src/yarn.lock index dac349289d14..9103833c2f2f 100644 --- a/presto-ui/src/yarn.lock +++ b/presto-ui/src/yarn.lock @@ -3152,9 +3152,9 @@ multicast-dns@^7.2.5: thunky "^1.0.2" nanoid@^3.3.6, nanoid@^3.3.7: - version "3.3.7" - resolved "https://registry.yarnpkg.com/nanoid/-/nanoid-3.3.7.tgz#d0c301a691bc8d54efa0a2226ccf3fe2fd656bd8" - integrity sha512-eSRppjcPIatRIMC1U6UngP8XFcz8MQWGQdt1MTBQ7NaAmvXDfvNxbvWV3x2y6CdEUciCSsDHDQZbhYaB8QEo2g== + version "3.3.8" + resolved "https://registry.yarnpkg.com/nanoid/-/nanoid-3.3.8.tgz#b1be3030bee36aaff18bacb375e5cce521684baf" + integrity sha512-WNLf5Sd8oZxOm+TzppcYk8gVOgP+l58xNy58D0nbUnOxOWRWvlcCV4kUF7ltmI6PsrLl/BgKEyS4mqsGChFN0w== negotiator@0.6.3: version "0.6.3" From 3b1aff96e3d6bcc9483a4aabe1aa3fdaf380b309 Mon Sep 17 00:00:00 2001 From: Reetika Agrawal Date: Thu, 12 Sep 2024 18:19:19 +0530 Subject: [PATCH 16/25] Set splitsize for hadoop InputFormat to Presto max_split_size --- .../presto/hive/StoragePartitionLoader.java | 4 + .../presto/hive/TestCustomInputSplits.java | 80 +++++++++++++++++++ 2 files changed, 84 insertions(+) create mode 100644 presto-hive/src/test/java/com/facebook/presto/hive/TestCustomInputSplits.java diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/StoragePartitionLoader.java b/presto-hive/src/main/java/com/facebook/presto/hive/StoragePartitionLoader.java index 08ab8a7b5de7..6a4052656ff4 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/StoragePartitionLoader.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/StoragePartitionLoader.java @@ -64,6 +64,7 @@ import static com.facebook.presto.hive.HiveErrorCode.HIVE_UNSUPPORTED_FORMAT; import static com.facebook.presto.hive.HiveMetadata.shouldCreateFilesForMissingBuckets; import static com.facebook.presto.hive.HiveSessionProperties.getMaxInitialSplitSize; +import static com.facebook.presto.hive.HiveSessionProperties.getMaxSplitSize; import static com.facebook.presto.hive.HiveSessionProperties.isFileSplittable; import static com.facebook.presto.hive.HiveSessionProperties.isOrderBasedExecutionEnabled; import static com.facebook.presto.hive.HiveSessionProperties.isSkipEmptyFilesEnabled; @@ -111,6 +112,7 @@ public class StoragePartitionLoader private final Deque> fileIterators; private final boolean schedulerUsesHostAddresses; private final boolean partialAggregationsPushedDown; + private static final String SPLIT_MINSIZE = "mapreduce.input.fileinputformat.split.minsize"; public StoragePartitionLoader( Table table, @@ -185,6 +187,7 @@ private ListenableFuture handleSymlinkTextInputFormat(ExtendedFileSystem fs, JobConf targetJob = toJobConf(targetFilesystem.getConf()); targetJob.setInputFormat(TextInputFormat.class); targetInputFormat.configure(targetJob); + targetJob.set(SPLIT_MINSIZE, Long.toString(getMaxSplitSize(session).toBytes())); FileInputFormat.setInputPaths(targetJob, targetPath); InputSplit[] targetSplits = targetInputFormat.getSplits(targetJob, 0); @@ -214,6 +217,7 @@ private ListenableFuture handleGetSplitsFromInputFormat(Configuration configu FileInputFormat.setInputPaths(jobConf, path); // SerDes parameters and Table parameters passing into input format fromProperties(schema).forEach(jobConf::set); + jobConf.set(SPLIT_MINSIZE, Long.toString(getMaxSplitSize(session).toBytes())); InputSplit[] splits = inputFormat.getSplits(jobConf, 0); return addSplitsToSource(splits, splitFactory, hiveSplitSource, stopped); diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestCustomInputSplits.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestCustomInputSplits.java new file mode 100644 index 000000000000..5f6c9061080f --- /dev/null +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestCustomInputSplits.java @@ -0,0 +1,80 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.hive; + +import com.google.common.io.Resources; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.HadoopExtendedFileSystem; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.FileInputFormat; +import org.apache.hadoop.mapred.InputSplit; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.TextInputFormat; +import org.testng.annotations.Test; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; + +@Test(singleThreaded = true) +public class TestCustomInputSplits +{ + private static final String SPLIT_MINSIZE = "mapreduce.input.fileinputformat.split.minsize"; + private static final String LOCAL_BLOCK_SIZE = "fs.local.block.size"; + private static final String DFS_BLOCK_SIZE = "dfs.blocksize"; + + @Test + public void testCustomSplitSize() throws Exception + { + long splitSize = 1000; + long blockSize = 500; + String filePath = Resources.getResource("addressbook.parquet").getFile(); + Path path = new Path(filePath); + + TextInputFormat targetInputFormat = new DummyTextInputFormat(); + JobConf jobConf = new JobConf(); + FileInputFormat.setInputPaths(jobConf, path); + + jobConf.set("fs.defaultFS", "file:///"); + jobConf.setClass("fs.file.impl", TestingFileSystem.class, FileSystem.class); + jobConf.setClass("fs.hdfs.impl", TestingFileSystem.class, FileSystem.class); + jobConf.setBoolean("fs.file.impl.disable.cache", true); + jobConf.setLong(LOCAL_BLOCK_SIZE, blockSize); + jobConf.setLong(DFS_BLOCK_SIZE, blockSize); + jobConf.set(SPLIT_MINSIZE, Long.toString(splitSize)); + + InputSplit[] targetSplits = targetInputFormat.getSplits(jobConf, 0); + + assertNotNull(targetSplits); + assertEquals(targetSplits.length, 4); + } + + public class DummyTextInputFormat + extends TextInputFormat + { + protected boolean isSplitable(FileSystem fs, Path file) + { + return true; + } + } + + public static class TestingFileSystem + extends HadoopExtendedFileSystem + { + public TestingFileSystem() + { + super(new LocalFileSystem()); + } + } +} From 8136f7cc230893358b86b5af5c13eb4763b5655f Mon Sep 17 00:00:00 2001 From: Sergey Pershin Date: Wed, 18 Dec 2024 15:17:28 -0800 Subject: [PATCH 17/25] Add "native_expression.max_compiled_regexes" session property. --- .../src/main/sphinx/presto_cpp/properties-session.rst | 11 ++++++++++- .../NativeWorkerSessionPropertyProvider.java | 7 +++++++ .../presto_cpp/main/SessionProperties.cpp | 9 +++++++++ .../presto_cpp/main/SessionProperties.h | 5 +++++ .../presto_cpp/main/tests/QueryContextManagerTest.cpp | 2 ++ 5 files changed, 33 insertions(+), 1 deletion(-) diff --git a/presto-docs/src/main/sphinx/presto_cpp/properties-session.rst b/presto-docs/src/main/sphinx/presto_cpp/properties-session.rst index b699b922003a..47977a0627d2 100644 --- a/presto-docs/src/main/sphinx/presto_cpp/properties-session.rst +++ b/presto-docs/src/main/sphinx/presto_cpp/properties-session.rst @@ -179,7 +179,7 @@ Native Execution only. Enable row number spilling on native engine. Native Execution only. Enable simplified path in expression evaluation. ``native_expression_max_array_size_in_reduce`` -^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ * **Type:** ``integer`` * **Default value:** ``100000`` @@ -187,6 +187,15 @@ Native Execution only. Enable simplified path in expression evaluation. Native Execution only. The `reduce `_ function will throw an error if it encounters an array of size greater than this value. +``native_expression_max_compiled_regexes`` +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +* **Type:** ``integer`` +* **Default value:** ``100`` + +Native Execution only. Controls maximum number of compiled regular expression patterns per +regular expression function instance per thread of execution. + ``native_spill_compression_codec`` ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ diff --git a/presto-main/src/main/java/com/facebook/presto/sessionpropertyproviders/NativeWorkerSessionPropertyProvider.java b/presto-main/src/main/java/com/facebook/presto/sessionpropertyproviders/NativeWorkerSessionPropertyProvider.java index 08d6c939b6d8..caf3de920c3f 100644 --- a/presto-main/src/main/java/com/facebook/presto/sessionpropertyproviders/NativeWorkerSessionPropertyProvider.java +++ b/presto-main/src/main/java/com/facebook/presto/sessionpropertyproviders/NativeWorkerSessionPropertyProvider.java @@ -34,6 +34,7 @@ public class NativeWorkerSessionPropertyProvider { public static final String NATIVE_SIMPLIFIED_EXPRESSION_EVALUATION_ENABLED = "native_simplified_expression_evaluation_enabled"; public static final String NATIVE_EXPRESSION_MAX_ARRAY_SIZE_IN_REDUCE = "native_expression_max_array_size_in_reduce"; + public static final String NATIVE_EXPRESSION_MAX_COMPILED_REGEXES = "native_expression_max_compiled_regexes"; public static final String NATIVE_MAX_SPILL_LEVEL = "native_max_spill_level"; public static final String NATIVE_MAX_SPILL_FILE_SIZE = "native_max_spill_file_size"; public static final String NATIVE_SPILL_COMPRESSION_CODEC = "native_spill_compression_codec"; @@ -90,6 +91,12 @@ public NativeWorkerSessionPropertyProvider(FeaturesConfig featuresConfig) "Native Execution only. Reduce() function will throw an error if it encounters an array of size greater than this value.", 100000, !nativeExecution), + integerProperty( + NATIVE_EXPRESSION_MAX_COMPILED_REGEXES, + "Native Execution only. Controls maximum number of compiled regular expression patterns " + + "per regular expression function instance per thread of execution.", + 100, + !nativeExecution), integerProperty( NATIVE_MAX_SPILL_LEVEL, "Native Execution only. The maximum allowed spilling level for hash join build.\n" + diff --git a/presto-native-execution/presto_cpp/main/SessionProperties.cpp b/presto-native-execution/presto_cpp/main/SessionProperties.cpp index 30ef49e3420b..314e81c3110f 100644 --- a/presto-native-execution/presto_cpp/main/SessionProperties.cpp +++ b/presto-native-execution/presto_cpp/main/SessionProperties.cpp @@ -72,6 +72,15 @@ SessionProperties::SessionProperties() { QueryConfig::kExprMaxArraySizeInReduce, std::to_string(c.exprMaxArraySizeInReduce())); + addSessionProperty( + kExprMaxCompiledRegexes, + "Controls maximum number of compiled regular expression patterns per regular expression function instance " + "per thread of execution.", + BIGINT(), + false, + QueryConfig::kExprMaxCompiledRegexes, + std::to_string(c.exprMaxCompiledRegexes())); + addSessionProperty( kMaxPartialAggregationMemory, "The max partial aggregation memory when data reduction is not optimal.", diff --git a/presto-native-execution/presto_cpp/main/SessionProperties.h b/presto-native-execution/presto_cpp/main/SessionProperties.h index 3156c66de756..a952c052845b 100644 --- a/presto-native-execution/presto_cpp/main/SessionProperties.h +++ b/presto-native-execution/presto_cpp/main/SessionProperties.h @@ -82,6 +82,11 @@ class SessionProperties { static constexpr const char* kExprMaxArraySizeInReduce = "native_expression_max_array_size_in_reduce"; + /// Controls maximum number of compiled regular expression patterns per + /// regular expression function instance per thread of execution. + static constexpr const char* kExprMaxCompiledRegexes = + "native_expression_max_compiled_regexes"; + /// The maximum memory used by partial aggregation when data reduction is not /// optimal. static constexpr const char* kMaxPartialAggregationMemory = diff --git a/presto-native-execution/presto_cpp/main/tests/QueryContextManagerTest.cpp b/presto-native-execution/presto_cpp/main/tests/QueryContextManagerTest.cpp index 7a686d33d179..0f258c589639 100644 --- a/presto-native-execution/presto_cpp/main/tests/QueryContextManagerTest.cpp +++ b/presto-native-execution/presto_cpp/main/tests/QueryContextManagerTest.cpp @@ -59,6 +59,7 @@ TEST_F(QueryContextManagerTest, nativeSessionProperties) { {"native_selective_nimble_reader_enabled", "true"}, {"aggregation_spill_all", "true"}, {"native_expression_max_array_size_in_reduce", "99999"}, + {"native_expression_max_compiled_regexes", "54321"}, }}; auto queryCtx = taskManager_->getQueryContextManager()->findOrCreateQueryCtx( taskId, session); @@ -73,6 +74,7 @@ TEST_F(QueryContextManagerTest, nativeSessionProperties) { EXPECT_TRUE(queryCtx->queryConfig().selectiveNimbleReaderEnabled()); EXPECT_EQ(queryCtx->queryConfig().spillWriteBufferSize(), 1024); EXPECT_EQ(queryCtx->queryConfig().exprMaxArraySizeInReduce(), 99999); + EXPECT_EQ(queryCtx->queryConfig().exprMaxCompiledRegexes(), 54321); } TEST_F(QueryContextManagerTest, defaultSessionProperties) { From 74c3c46c361392ba53b18a496694f3b3c34a8c64 Mon Sep 17 00:00:00 2001 From: Andrii Rosa Date: Thu, 5 Dec 2024 07:31:15 -0800 Subject: [PATCH 18/25] [native] Regenerate Presto protocol --- .../presto_protocol/core/presto_protocol_core.cpp | 14 ++++++++++++++ .../presto_protocol/core/presto_protocol_core.h | 1 + 2 files changed, 15 insertions(+) diff --git a/presto-native-execution/presto_cpp/presto_protocol/core/presto_protocol_core.cpp b/presto-native-execution/presto_cpp/presto_protocol/core/presto_protocol_core.cpp index 60b9a0a601e6..3cfabe64627f 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/core/presto_protocol_core.cpp +++ b/presto-native-execution/presto_cpp/presto_protocol/core/presto_protocol_core.cpp @@ -8690,6 +8690,13 @@ void to_json(json& j, const SortNode& p) { "OrderingScheme", "orderingScheme"); to_json_key(j, "isPartial", p.isPartial, "SortNode", "bool", "isPartial"); + to_json_key( + j, + "partitionBy", + p.partitionBy, + "SortNode", + "List", + "partitionBy"); } void from_json(const json& j, SortNode& p) { @@ -8704,6 +8711,13 @@ void from_json(const json& j, SortNode& p) { "OrderingScheme", "orderingScheme"); from_json_key(j, "isPartial", p.isPartial, "SortNode", "bool", "isPartial"); + from_json_key( + j, + "partitionBy", + p.partitionBy, + "SortNode", + "List", + "partitionBy"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { diff --git a/presto-native-execution/presto_cpp/presto_protocol/core/presto_protocol_core.h b/presto-native-execution/presto_cpp/presto_protocol/core/presto_protocol_core.h index 902c0c24ce5a..a50e7da4839e 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/core/presto_protocol_core.h +++ b/presto-native-execution/presto_cpp/presto_protocol/core/presto_protocol_core.h @@ -2025,6 +2025,7 @@ struct SortNode : public PlanNode { std::shared_ptr source = {}; OrderingScheme orderingScheme = {}; bool isPartial = {}; + List partitionBy = {}; SortNode() noexcept; }; From 0720f1784b1d1f8c52ba586f5031e55ccb308031 Mon Sep 17 00:00:00 2001 From: Andrii Rosa Date: Tue, 26 Nov 2024 09:58:55 -0800 Subject: [PATCH 19/25] [native] Support partitioned writer scaling --- .../facebook/presto/hive/HiveMetadata.java | 16 +- .../hive/TestHiveIntegrationSmokeTest.java | 1 + .../presto/SystemSessionProperties.java | 12 +- .../presto/sql/TemporaryTableUtil.java | 1 + .../presto/sql/analyzer/FeaturesConfig.java | 13 + .../sql/planner/BasePlanFragmenter.java | 5 +- .../sql/planner/LocalExecutionPlanner.java | 4 +- .../presto/sql/planner/LogicalPlanner.java | 4 +- .../sql/planner/PlanFragmenterUtils.java | 1 + ...PushPartialAggregationThroughExchange.java | 1 + .../rule/PushProjectionThroughExchange.java | 1 + ...shRemoteExchangeThroughAssignUniqueId.java | 1 + .../PushRemoteExchangeThroughGroupId.java | 1 + .../rule/PushTableWriteThroughUnion.java | 2 +- .../planner/optimizations/AddExchanges.java | 10 +- .../optimizations/AddLocalExchanges.java | 187 ++++++------- .../HashGenerationOptimizer.java | 1 + .../MergePartialAggregationsWithFilter.java | 1 + .../optimizations/PartitioningUtils.java | 9 +- .../optimizations/PropertyDerivations.java | 19 +- .../PruneUnreferencedOutputs.java | 1 + .../StreamPropertyDerivations.java | 4 +- .../planner/optimizations/SymbolMapper.java | 1 + .../UnaliasSymbolReferences.java | 1 + .../presto/sql/planner/plan/ExchangeNode.java | 1 + .../sql/analyzer/TestFeaturesConfig.java | 7 +- .../planner/TestCanonicalPlanGenerator.java | 2 +- .../main/types/PrestoToVeloxQueryPlan.cpp | 20 +- .../main/types/tests/data/FinalAgg.json | 2 + .../main/types/tests/data/OffsetLimit.json | 4 + .../main/types/tests/data/Output.json | 1 + .../types/tests/data/PartitionedOutput.json | 1 + .../main/types/tests/data/ScanAgg.json | 1 + .../main/types/tests/data/ScanAggBatch.json | 1 + .../tests/data/ScanAggCustomConnectorId.json | 1 + .../main/types/tests/data/ValuesPipeTest.json | 2 + .../core/presto_protocol_core.cpp | 14 + .../core/presto_protocol_core.h | 1 + .../tests/data/ExchangeNode.json | 1 + .../tests/data/FilterNode.json | 1 + .../tests/data/OutputNode.json | 1 + .../nativeworker/TestPrestoNativeWriter.java | 245 +++++++++++++++++- .../presto/spi/ConnectorNewTableLayout.java | 18 +- .../facebook/presto/spi/NewTableLayout.java | 5 + .../spi/PartitionedTableWritePolicy.java | 20 ++ .../presto/spi/plan/PartitioningScheme.java | 45 +++- .../presto/spi/plan/TableWriterNode.java | 5 + 47 files changed, 572 insertions(+), 124 deletions(-) create mode 100644 presto-spi/src/main/java/com/facebook/presto/spi/PartitionedTableWritePolicy.java diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java index 5c38af670a6b..3d3a0eacd4d4 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java @@ -333,6 +333,8 @@ import static com.facebook.presto.spi.MaterializedViewStatus.MaterializedViewState.NOT_MATERIALIZED; import static com.facebook.presto.spi.MaterializedViewStatus.MaterializedViewState.PARTIALLY_MATERIALIZED; import static com.facebook.presto.spi.MaterializedViewStatus.MaterializedViewState.TOO_MANY_PARTITIONS_MISSING; +import static com.facebook.presto.spi.PartitionedTableWritePolicy.MULTIPLE_WRITERS_PER_PARTITION_ALLOWED; +import static com.facebook.presto.spi.PartitionedTableWritePolicy.SINGLE_WRITER_PER_PARTITION_REQUIRED; import static com.facebook.presto.spi.StandardErrorCode.ALREADY_EXISTS; import static com.facebook.presto.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; import static com.facebook.presto.spi.StandardErrorCode.INVALID_ANALYZE_PROPERTY; @@ -2995,7 +2997,7 @@ public Optional getInsertLayout(ConnectorSession sessio Optional hiveBucketHandle = getHiveBucketHandle(session, table); if (!hiveBucketHandle.isPresent()) { - if (!isShufflePartitionedColumnsForTableWriteEnabled(session) || table.getPartitionColumns().isEmpty()) { + if (table.getPartitionColumns().isEmpty()) { return Optional.empty(); } @@ -3010,7 +3012,10 @@ public Optional getInsertLayout(ConnectorSession sessio .map(Column::getName) .collect(toList()); - return Optional.of(new ConnectorNewTableLayout(partitioningHandle, partitionedBy)); + return Optional.of(new ConnectorNewTableLayout( + partitioningHandle, + partitionedBy, + isShufflePartitionedColumnsForTableWriteEnabled(session) ? SINGLE_WRITER_PER_PARTITION_REQUIRED : MULTIPLE_WRITERS_PER_PARTITION_ALLOWED)); } HiveBucketProperty bucketProperty = table.getStorage().getBucketProperty() .orElseThrow(() -> new NoSuchElementException("Bucket property should be set")); @@ -3055,7 +3060,7 @@ public Optional getNewTableLayout(ConnectorSession sess Optional bucketProperty = getBucketProperty(tableMetadata.getProperties()); if (!bucketProperty.isPresent()) { List partitionedBy = getPartitionedBy(tableMetadata.getProperties()); - if (!isShufflePartitionedColumnsForTableWriteEnabled(session) || partitionedBy.isEmpty()) { + if (partitionedBy.isEmpty()) { return Optional.empty(); } @@ -3074,7 +3079,10 @@ public Optional getNewTableLayout(ConnectorSession sess .collect(toList()), OptionalInt.empty()); - return Optional.of(new ConnectorNewTableLayout(partitioningHandle, partitionedBy)); + return Optional.of(new ConnectorNewTableLayout( + partitioningHandle, + partitionedBy, + isShufflePartitionedColumnsForTableWriteEnabled(session) ? SINGLE_WRITER_PER_PARTITION_REQUIRED : MULTIPLE_WRITERS_PER_PARTITION_ALLOWED)); } checkArgument(bucketProperty.get().getBucketFunctionType().equals(BucketFunctionType.HIVE_COMPATIBLE), "bucketFunctionType is expected to be HIVE_COMPATIBLE, got: %s", diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveIntegrationSmokeTest.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveIntegrationSmokeTest.java index 4b10aa81a45a..fa2a064dc958 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveIntegrationSmokeTest.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveIntegrationSmokeTest.java @@ -2296,6 +2296,7 @@ public void testBucketPruning() { Session session = getSession(); QueryRunner queryRunner = getQueryRunner(); + queryRunner.execute("CREATE TABLE orders_bucketed WITH (bucket_count = 11, bucketed_by = ARRAY['orderkey']) AS " + "SELECT * FROM orders"); diff --git a/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java b/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java index 8de9af7667eb..7f23e8172e80 100644 --- a/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java +++ b/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java @@ -335,6 +335,7 @@ public final class SystemSessionProperties public static final String NATIVE_EXECUTION_PROCESS_REUSE_ENABLED = "native_execution_process_reuse_enabled"; public static final String NATIVE_MIN_COLUMNAR_ENCODING_CHANNELS_TO_PREFER_ROW_WISE_ENCODING = "native_min_columnar_encoding_channels_to_prefer_row_wise_encoding"; public static final String NATIVE_ENFORCE_JOIN_BUILD_INPUT_PARTITION = "native_enforce_join_build_input_partition"; + public static final String NATIVE_EXECUTION_SCALE_WRITER_THREADS_ENABLED = "native_execution_scale_writer_threads_enabled"; private final List> sessionProperties; @@ -1841,7 +1842,11 @@ public SystemSessionProperties( SINGLE_NODE_EXECUTION_ENABLED, "Enable single node execution", featuresConfig.isSingleNodeExecutionEnabled(), - false)); + false), + booleanProperty(NATIVE_EXECUTION_SCALE_WRITER_THREADS_ENABLED, + "Enable automatic scaling of writer threads", + featuresConfig.isNativeExecutionScaleWritersThreadsEnabled(), + !featuresConfig.isNativeExecutionEnabled())); } public static boolean isSpoolingOutputBufferEnabled(Session session) @@ -3132,4 +3137,9 @@ public static int getMinColumnarEncodingChannelsToPreferRowWiseEncoding(Session { return session.getSystemProperty(NATIVE_MIN_COLUMNAR_ENCODING_CHANNELS_TO_PREFER_ROW_WISE_ENCODING, Integer.class); } + + public static boolean isNativeExecutionScaleWritersThreadsEnabled(Session session) + { + return session.getSystemProperty(NATIVE_EXECUTION_SCALE_WRITER_THREADS_ENABLED, Boolean.class); + } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/TemporaryTableUtil.java b/presto-main/src/main/java/com/facebook/presto/sql/TemporaryTableUtil.java index f40bc95a8d0a..d9c68ced227b 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/TemporaryTableUtil.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/TemporaryTableUtil.java @@ -288,6 +288,7 @@ public static TableFinishNode createTemporaryTableWriteWithExchanges( outputs, Optional.empty(), false, + false, COLUMNAR, Optional.empty()); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java index 55a812e7054d..d6c329d195d9 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java @@ -291,6 +291,7 @@ public class FeaturesConfig private boolean prestoSparkExecutionEnvironment; private boolean singleNodeExecutionEnabled; + private boolean nativeExecutionScaleWritersThreadsEnabled; public enum PartitioningPrecisionStrategy { @@ -2889,4 +2890,16 @@ public FeaturesConfig setSingleNodeExecutionEnabled(boolean singleNodeExecutionE this.singleNodeExecutionEnabled = singleNodeExecutionEnabled; return this; } + + public boolean isNativeExecutionScaleWritersThreadsEnabled() + { + return nativeExecutionScaleWritersThreadsEnabled; + } + + @Config("native-execution-scale-writer-threads-enabled") + public FeaturesConfig setNativeExecutionScaleWritersThreadsEnabled(boolean nativeExecutionScaleWritersThreadsEnabled) + { + this.nativeExecutionScaleWritersThreadsEnabled = nativeExecutionScaleWritersThreadsEnabled; + return this; + } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/BasePlanFragmenter.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/BasePlanFragmenter.java index 190271f3e39a..b31906af0df5 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/BasePlanFragmenter.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/BasePlanFragmenter.java @@ -257,7 +257,7 @@ public PlanNode visitTableScan(TableScanNode node, RewriteContext context) { - if (node.getTablePartitioningScheme().isPresent()) { + if (node.isSingleWriterPerPartitionRequired()) { context.get().setDistribution(node.getTablePartitioningScheme().get().getPartitioning().getHandle(), metadata, session); } return context.defaultRewrite(node, context.get()); @@ -292,6 +292,7 @@ public PlanNode visitExchange(ExchangeNode exchange, RewriteContext context) { checkArgument(exchange.getScope() == REMOTE_STREAMING, "Unexpected exchange scope: %s", exchange.getScope()); + checkArgument(!exchange.getPartitioningScheme().isScaleWriters(), "task scaling for partitioned tables is not yet supported"); PartitioningScheme partitioningScheme = exchange.getPartitioningScheme(); @@ -338,6 +339,8 @@ private PlanNode createRemoteMaterializedExchange(ExchangeNode exchange, Rewrite checkArgument(exchange.getType() == REPARTITION, "Unexpected exchange type: %s", exchange.getType()); checkArgument(exchange.getScope() == REMOTE_MATERIALIZED, "Unexpected exchange scope: %s", exchange.getScope()); + checkArgument(!exchange.getPartitioningScheme().isScaleWriters(), "task scaling for partitioned tables is not yet supported"); + PartitioningScheme partitioningScheme = exchange.getPartitioningScheme(); PartitioningHandle partitioningHandle = partitioningScheme.getPartitioning().getHandle(); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/LocalExecutionPlanner.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/LocalExecutionPlanner.java index fbde1ea09421..b2254ec86c31 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/LocalExecutionPlanner.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/LocalExecutionPlanner.java @@ -2680,7 +2680,7 @@ public PhysicalOperation visitSemiJoin(SemiJoinNode node, LocalExecutionPlanCont public PhysicalOperation visitTableWriter(TableWriterNode node, LocalExecutionPlanContext context) { // Set table writer count - if (node.getTablePartitioningScheme().isPresent()) { + if (node.isSingleWriterPerPartitionRequired()) { context.setDriverInstanceCount(getTaskPartitionedWriterCount(session)); } else { @@ -3070,6 +3070,8 @@ private PhysicalOperation createLocalMerge(ExchangeNode node, LocalExecutionPlan private PhysicalOperation createLocalExchange(ExchangeNode node, LocalExecutionPlanContext context) { + checkArgument(!node.getPartitioningScheme().isScaleWriters(), "thread scaling for partitioned tables is only supported by native execution"); + int driverInstanceCount; if (node.getType() == ExchangeNode.Type.GATHER) { driverInstanceCount = 1; 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 9d0d1aaaf180..ccc459bc0ceb 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 @@ -90,6 +90,7 @@ import static com.facebook.presto.common.type.VarbinaryType.VARBINARY; import static com.facebook.presto.metadata.MetadataUtil.getConnectorIdOrThrow; import static com.facebook.presto.metadata.MetadataUtil.toSchemaTableName; +import static com.facebook.presto.spi.PartitionedTableWritePolicy.MULTIPLE_WRITERS_PER_PARTITION_ALLOWED; import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED; import static com.facebook.presto.spi.plan.AggregationNode.singleGroupingSet; import static com.facebook.presto.spi.plan.LimitNode.Step.FINAL; @@ -633,7 +634,8 @@ private static Optional getPartitioningSchemeForTableWrite(O partitioningScheme = Optional.of(new PartitioningScheme( Partitioning.create(tableLayout.get().getPartitioning(), partitionFunctionArguments), - outputLayout)); + outputLayout, + tableLayout.get().getWriterPolicy() == MULTIPLE_WRITERS_PER_PARTITION_ALLOWED)); } return partitioningScheme; } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanFragmenterUtils.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanFragmenterUtils.java index 2ad78df0be9c..90d1c80564a3 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanFragmenterUtils.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanFragmenterUtils.java @@ -228,6 +228,7 @@ private static SubPlan reassignPartitioningHandleIfNecessaryHelper(Metadata meta outputPartitioningScheme.getOutputLayout(), outputPartitioningScheme.getHashColumn(), outputPartitioningScheme.isReplicateNullsAndAny(), + outputPartitioningScheme.isScaleWriters(), outputPartitioningScheme.getEncoding(), outputPartitioningScheme.getBucketToPartition()), fragment.getStageExecutionDescriptor(), diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PushPartialAggregationThroughExchange.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PushPartialAggregationThroughExchange.java index d3bc4c8b562b..3c5f12fa3486 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PushPartialAggregationThroughExchange.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PushPartialAggregationThroughExchange.java @@ -233,6 +233,7 @@ private PlanNode pushPartial(AggregationNode aggregation, ExchangeNode exchange, aggregationOutputs, exchange.getPartitioningScheme().getHashColumn(), exchange.getPartitioningScheme().isReplicateNullsAndAny(), + exchange.getPartitioningScheme().isScaleWriters(), exchange.getPartitioningScheme().getEncoding(), exchange.getPartitioningScheme().getBucketToPartition()); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PushProjectionThroughExchange.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PushProjectionThroughExchange.java index cc568f94a01e..10071de89825 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PushProjectionThroughExchange.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PushProjectionThroughExchange.java @@ -146,6 +146,7 @@ public Result apply(ProjectNode project, Captures captures, Context context) outputBuilder.build(), exchange.getPartitioningScheme().getHashColumn(), exchange.getPartitioningScheme().isReplicateNullsAndAny(), + exchange.getPartitioningScheme().isScaleWriters(), exchange.getPartitioningScheme().getEncoding(), exchange.getPartitioningScheme().getBucketToPartition()); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PushRemoteExchangeThroughAssignUniqueId.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PushRemoteExchangeThroughAssignUniqueId.java index c98ddfe1c3e2..4a22d736e3e7 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PushRemoteExchangeThroughAssignUniqueId.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PushRemoteExchangeThroughAssignUniqueId.java @@ -81,6 +81,7 @@ public Result apply(ExchangeNode node, Captures captures, Context context) removeVariable(partitioningScheme.getOutputLayout(), assignUniqueId.getIdVariable()), partitioningScheme.getHashColumn(), partitioningScheme.isReplicateNullsAndAny(), + partitioningScheme.isScaleWriters(), partitioningScheme.getEncoding(), partitioningScheme.getBucketToPartition()), ImmutableList.of(assignUniqueId.getSource()), diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PushRemoteExchangeThroughGroupId.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PushRemoteExchangeThroughGroupId.java index 085dcc44f2f6..d25004efad1a 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PushRemoteExchangeThroughGroupId.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PushRemoteExchangeThroughGroupId.java @@ -152,6 +152,7 @@ public Result apply(ExchangeNode node, Captures captures, Context context) outputLayout, partitioningScheme.getHashColumn(), partitioningScheme.isReplicateNullsAndAny(), + partitioningScheme.isScaleWriters(), partitioningScheme.getEncoding(), partitioningScheme.getBucketToPartition()), ImmutableList.of(groupIdNode.getSource()), diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PushTableWriteThroughUnion.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PushTableWriteThroughUnion.java index 9df8e9d8963a..bb10b2185f7b 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PushTableWriteThroughUnion.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PushTableWriteThroughUnion.java @@ -52,7 +52,7 @@ public class PushTableWriteThroughUnion // guaranteed regardless of this optimizer. The level of local parallelism will be // determined by LocalExecutionPlanner separately, and shouldn't be a concern of // this optimizer. - .matching(tableWriter -> !tableWriter.getTablePartitioningScheme().isPresent()) + .matching(tableWriter -> !tableWriter.isSingleWriterPerPartitionRequired()) .with(source().matching(union().capturedAs(CHILD))); @Override diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/AddExchanges.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/AddExchanges.java index b099f7766621..460c5272fdc8 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/AddExchanges.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/AddExchanges.java @@ -737,13 +737,18 @@ public PlanWithProperties visitTableWriter(TableWriterNode node, PreferredProper PlanWithProperties source = accept(node.getSource(), preferredProperties); Optional shufflePartitioningScheme = node.getTablePartitioningScheme(); - if (!shufflePartitioningScheme.isPresent()) { + if (!node.isSingleWriterPerPartitionRequired()) { + // prefer scale writers if single writer per partition is not required + // TODO: take into account partitioning scheme in scale writer tasks implementation if (scaleWriters) { shufflePartitioningScheme = Optional.of(new PartitioningScheme(Partitioning.create(SCALED_WRITER_DISTRIBUTION, ImmutableList.of()), source.getNode().getOutputVariables())); } else if (redistributeWrites) { shufflePartitioningScheme = Optional.of(new PartitioningScheme(Partitioning.create(FIXED_ARBITRARY_DISTRIBUTION, ImmutableList.of()), source.getNode().getOutputVariables())); } + else { + return rebaseAndDeriveProperties(node, source); + } } if (shufflePartitioningScheme.isPresent() && @@ -1118,6 +1123,7 @@ public PlanWithProperties visitSemiJoin(SemiJoinNode node, PreferredProperties p filteringSource.getNode().getOutputVariables(), Optional.empty(), true, + false, COLUMNAR, Optional.empty())), filteringSource.getProperties()); @@ -1160,6 +1166,7 @@ public PlanWithProperties visitSemiJoin(SemiJoinNode node, PreferredProperties p filteringSource.getNode().getOutputVariables(), Optional.empty(), true, + false, COLUMNAR, Optional.empty())), filteringSource.getProperties()); @@ -1336,6 +1343,7 @@ public PlanWithProperties visitUnion(UnionNode node, PreferredProperties parentP source.getNode().getOutputVariables(), Optional.empty(), nullsAndAnyReplicated, + false, COLUMNAR, Optional.empty())), source.getProperties()); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/AddLocalExchanges.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/AddLocalExchanges.java index ebec1d015029..975130d4307d 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/AddLocalExchanges.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/AddLocalExchanges.java @@ -72,6 +72,7 @@ import static com.facebook.presto.SystemSessionProperties.isDistributedSortEnabled; import static com.facebook.presto.SystemSessionProperties.isEnforceFixedDistributionForOutputOperator; import static com.facebook.presto.SystemSessionProperties.isJoinSpillingEnabled; +import static com.facebook.presto.SystemSessionProperties.isNativeExecutionScaleWritersThreadsEnabled; import static com.facebook.presto.SystemSessionProperties.isNativeJoinBuildPartitionEnforced; import static com.facebook.presto.SystemSessionProperties.isQuickDistinctLimitEnabled; import static com.facebook.presto.SystemSessionProperties.isSegmentedAggregationEnabled; @@ -102,6 +103,7 @@ import static com.facebook.presto.sql.planner.plan.ExchangeNode.systemPartitionedExchange; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; +import static com.google.common.base.Verify.verify; import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableSet.toImmutableSet; import static java.util.Objects.requireNonNull; @@ -589,120 +591,121 @@ public PlanWithProperties visitTopNRowNumber(TopNRowNumberNode node, StreamPrefe // @Override - public PlanWithProperties visitTableWriter(TableWriterNode originalTableWriterNode, StreamPreferredProperties parentPreferences) + public PlanWithProperties visitTableWriter(TableWriterNode tableWrite, StreamPreferredProperties parentPreferences) { - if (originalTableWriterNode.getTablePartitioningScheme().isPresent() && getTaskPartitionedWriterCount(session) == 1) { - return planAndEnforceChildren(originalTableWriterNode, singleStream(), defaultParallelism(session)); + // When table is partitioned and single writer per partition is required (for example a bucketed table in Hive connector) + if (tableWrite.isSingleWriterPerPartitionRequired()) { + // special case when a single table writer per task is requested + if (getTaskPartitionedWriterCount(session) == 1) { + return planAndEnforceChildren(tableWrite, singleStream(), defaultParallelism(session)); + } + PlanWithProperties source = accept(tableWrite.getSource(), defaultParallelism(session)); + PlanWithProperties exchange = deriveProperties( + partitionedExchange( + idAllocator.getNextId(), + LOCAL, + source.getNode(), + tableWrite.getTablePartitioningScheme().get()), + source.getProperties()); + return planTableWriteWithTableWriteMerge(tableWrite, exchange); } - if (!originalTableWriterNode.getTablePartitioningScheme().isPresent() && getTaskWriterCount(session) == 1) { - return planAndEnforceChildren(originalTableWriterNode, singleStream(), defaultParallelism(session)); + // special case when a single table writer per task is requested + if (getTaskWriterCount(session) == 1) { + return planAndEnforceChildren(tableWrite, singleStream(), defaultParallelism(session)); } - Optional statisticAggregations = originalTableWriterNode - .getStatisticsAggregation() - .map(aggregations -> splitIntoPartialAndIntermediate( - aggregations, - variableAllocator, - metadata.getFunctionAndTypeManager())); - - PlanWithProperties tableWriter; - - if (!originalTableWriterNode.getTablePartitioningScheme().isPresent()) { - int taskWriterCount = getTaskWriterCount(session); - int taskConcurrency = getTaskConcurrency(session); - if (taskWriterCount == taskConcurrency) { - tableWriter = planAndEnforceChildren( - new TableWriterNode( - originalTableWriterNode.getSourceLocation(), - originalTableWriterNode.getId(), - originalTableWriterNode.getStatsEquivalentPlanNode(), - originalTableWriterNode.getSource(), - originalTableWriterNode.getTarget(), - variableAllocator.newVariable("partialrowcount", BIGINT), - variableAllocator.newVariable("partialfragments", VARBINARY), - variableAllocator.newVariable("partialcontext", VARBINARY), - originalTableWriterNode.getColumns(), - originalTableWriterNode.getColumnNames(), - originalTableWriterNode.getNotNullColumnVariables(), - originalTableWriterNode.getTablePartitioningScheme(), - statisticAggregations.map(StatisticAggregations.Parts::getPartialAggregation), - originalTableWriterNode.getTaskCountIfScaledWriter(), - originalTableWriterNode.getIsTemporaryTableWriter()), - fixedParallelism(), - fixedParallelism()); + // Writer thread scaling enabled and the output table allows multiple writers per partition (for example non bucketed table in Hive connector) + if (nativeExecution && isNativeExecutionScaleWritersThreadsEnabled(session)) { + PlanWithProperties source = accept(tableWrite.getSource(), defaultParallelism(session)); + PartitioningScheme partitioningScheme; + if (tableWrite.getTablePartitioningScheme().isPresent()) { + // Partitioning scheme is present and more than a single writer per partition is allowed (for example when table is not bucketed but partitioned in Hive connector) + partitioningScheme = tableWrite.getTablePartitioningScheme().get(); + verify(partitioningScheme.isScaleWriters()); } else { - PlanWithProperties source = accept(originalTableWriterNode.getSource(), defaultParallelism(session)); - PlanWithProperties exchange = deriveProperties( - roundRobinExchange(idAllocator.getNextId(), LOCAL, source.getNode()), - source.getProperties()); - tableWriter = deriveProperties( - new TableWriterNode( - originalTableWriterNode.getSourceLocation(), - originalTableWriterNode.getId(), - originalTableWriterNode.getStatsEquivalentPlanNode(), - exchange.getNode(), - originalTableWriterNode.getTarget(), - variableAllocator.newVariable("partialrowcount", BIGINT), - variableAllocator.newVariable("partialfragments", VARBINARY), - variableAllocator.newVariable("partialcontext", VARBINARY), - originalTableWriterNode.getColumns(), - originalTableWriterNode.getColumnNames(), - originalTableWriterNode.getNotNullColumnVariables(), - originalTableWriterNode.getTablePartitioningScheme(), - statisticAggregations.map(StatisticAggregations.Parts::getPartialAggregation), - originalTableWriterNode.getTaskCountIfScaledWriter(), - originalTableWriterNode.getIsTemporaryTableWriter()), - exchange.getProperties()); + // When partitioning scheme is not present (for example when table is not partitioned and not bucketed in Hive connector) + partitioningScheme = new PartitioningScheme( + Partitioning.create(FIXED_ARBITRARY_DISTRIBUTION, ImmutableList.of()), + source.getNode().getOutputVariables(), + true); } - } - else { - PlanWithProperties source = accept(originalTableWriterNode.getSource(), defaultParallelism(session)); PlanWithProperties exchange = deriveProperties( partitionedExchange( idAllocator.getNextId(), LOCAL, source.getNode(), - originalTableWriterNode.getTablePartitioningScheme().get()), + partitioningScheme), source.getProperties()); - tableWriter = deriveProperties( - new TableWriterNode( - originalTableWriterNode.getSourceLocation(), - originalTableWriterNode.getId(), - originalTableWriterNode.getStatsEquivalentPlanNode(), - exchange.getNode(), - originalTableWriterNode.getTarget(), - variableAllocator.newVariable("partialrowcount", BIGINT), - variableAllocator.newVariable("partialfragments", VARBINARY), - variableAllocator.newVariable("partialcontext", VARBINARY), - originalTableWriterNode.getColumns(), - originalTableWriterNode.getColumnNames(), - originalTableWriterNode.getNotNullColumnVariables(), - originalTableWriterNode.getTablePartitioningScheme(), - statisticAggregations.map(StatisticAggregations.Parts::getPartialAggregation), - originalTableWriterNode.getTaskCountIfScaledWriter(), - originalTableWriterNode.getIsTemporaryTableWriter()), - exchange.getProperties()); - } - - PlanWithProperties gatheringExchange = deriveProperties( + return planTableWriteWithTableWriteMerge(tableWrite, exchange); + } + + // Writer thread scaling is disabled and there is no strict partitioning requirement + int taskWriterCount = getTaskWriterCount(session); + int taskConcurrency = getTaskConcurrency(session); + if (taskWriterCount == taskConcurrency) { + // When table write concurrency is equal to task concurrency do not add en extra local exchange for improved efficiency + return planTableWriteWithTableWriteMerge( + tableWrite, + // When source distribution is MULTIPLE (for example a TableScan) add an exchange to achieve a fixed number of writer threads + planAndEnforce(tableWrite.getSource(), fixedParallelism(), fixedParallelism())); + } + else { + // When concurrency settings are different add an exchange to achieve a specific level of parallelism for table write + PlanWithProperties source = accept(tableWrite.getSource(), defaultParallelism(session)); + PlanWithProperties exchange = deriveProperties( + roundRobinExchange(idAllocator.getNextId(), LOCAL, source.getNode()), + source.getProperties()); + return planTableWriteWithTableWriteMerge(tableWrite, exchange); + } + } + + private PlanWithProperties planTableWriteWithTableWriteMerge(TableWriterNode tableWrite, PlanWithProperties source) + { + Optional statisticAggregations = tableWrite + .getStatisticsAggregation() + .map(aggregations -> splitIntoPartialAndIntermediate( + aggregations, + variableAllocator, + metadata.getFunctionAndTypeManager())); + + PlanWithProperties tableWriteWithProperties = deriveProperties( + new TableWriterNode( + tableWrite.getSourceLocation(), + tableWrite.getId(), + tableWrite.getStatsEquivalentPlanNode(), + source.getNode(), + tableWrite.getTarget(), + variableAllocator.newVariable("partialrowcount", BIGINT), + variableAllocator.newVariable("partialfragments", VARBINARY), + variableAllocator.newVariable("partialcontext", VARBINARY), + tableWrite.getColumns(), + tableWrite.getColumnNames(), + tableWrite.getNotNullColumnVariables(), + tableWrite.getTablePartitioningScheme(), + statisticAggregations.map(StatisticAggregations.Parts::getPartialAggregation), + tableWrite.getTaskCountIfScaledWriter(), + tableWrite.getIsTemporaryTableWriter()), + source.getProperties()); + + PlanWithProperties gatherExchangeWithProperties = deriveProperties( gatheringExchange( idAllocator.getNextId(), LOCAL, - tableWriter.getNode()), - tableWriter.getProperties()); + tableWriteWithProperties.getNode()), + tableWriteWithProperties.getProperties()); return deriveProperties( new TableWriterMergeNode( - originalTableWriterNode.getSourceLocation(), + tableWrite.getSourceLocation(), idAllocator.getNextId(), - gatheringExchange.getNode(), - originalTableWriterNode.getRowCountVariable(), - originalTableWriterNode.getFragmentVariable(), - originalTableWriterNode.getTableCommitContextVariable(), + gatherExchangeWithProperties.getNode(), + tableWrite.getRowCountVariable(), + tableWrite.getFragmentVariable(), + tableWrite.getTableCommitContextVariable(), statisticAggregations.map(StatisticAggregations.Parts::getIntermediateAggregation)), - gatheringExchange.getProperties()); + gatherExchangeWithProperties.getProperties()); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/HashGenerationOptimizer.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/HashGenerationOptimizer.java index 995a687c98b2..4c5a4b5fa65f 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/HashGenerationOptimizer.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/HashGenerationOptimizer.java @@ -587,6 +587,7 @@ public PlanWithProperties visitExchange(ExchangeNode node, HashComputationSet pa .build(), partitionVariables.map(newHashVariables::get), partitioningScheme.isReplicateNullsAndAny(), + partitioningScheme.isScaleWriters(), partitioningScheme.getEncoding(), partitioningScheme.getBucketToPartition()); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/MergePartialAggregationsWithFilter.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/MergePartialAggregationsWithFilter.java index 8db400fb2b98..1c6b974853ed 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/MergePartialAggregationsWithFilter.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/MergePartialAggregationsWithFilter.java @@ -356,6 +356,7 @@ public PlanNode visitExchange(ExchangeNode node, RewriteContext context children.get(children.size() - 1).getOutputVariables(), node.getPartitioningScheme().getHashColumn(), node.getPartitioningScheme().isReplicateNullsAndAny(), + node.getPartitioningScheme().isScaleWriters(), node.getPartitioningScheme().getEncoding(), node.getPartitioningScheme().getBucketToPartition()); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PartitioningUtils.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PartitioningUtils.java index 4cbd60d20c47..e193cbc62f77 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PartitioningUtils.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PartitioningUtils.java @@ -344,7 +344,14 @@ public static PartitioningScheme translateOutputLayout(PartitioningScheme partit .map(oldOutputLayout::indexOf) .map(newOutputLayout::get); - return new PartitioningScheme(newPartitioning, newOutputLayout, newHashSymbol, partitioningScheme.isReplicateNullsAndAny(), partitioningScheme.getEncoding(), partitioningScheme.getBucketToPartition()); + return new PartitioningScheme( + newPartitioning, + newOutputLayout, + newHashSymbol, + partitioningScheme.isReplicateNullsAndAny(), + partitioningScheme.isScaleWriters(), + partitioningScheme.getEncoding(), + partitioningScheme.getBucketToPartition()); } // Translates VariableReferenceExpression in arguments according to translator, keeps other arguments unchanged. diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PropertyDerivations.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PropertyDerivations.java index 3b00ebd61a33..0cacb674285e 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PropertyDerivations.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PropertyDerivations.java @@ -634,14 +634,23 @@ else if (inputProperties.stream().anyMatch(ActualProperties::isSingleNode)) { .local(localProperties.build()) .constants(constants) .build(); - case REPARTITION: + case REPARTITION: { + Global globalPartitioning; + if (node.getPartitioningScheme().isScaleWriters()) { + // no strict partitioning guarantees when multiple writers per partitions allowed (scaled writers) + globalPartitioning = arbitraryPartition(); + } + else { + globalPartitioning = partitionedOn( + node.getPartitioningScheme().getPartitioning(), + Optional.of(node.getPartitioningScheme().getPartitioning())) + .withReplicatedNulls(node.getPartitioningScheme().isReplicateNullsAndAny()); + } return ActualProperties.builder() - .global(partitionedOn( - node.getPartitioningScheme().getPartitioning(), - Optional.of(node.getPartitioningScheme().getPartitioning())) - .withReplicatedNulls(node.getPartitioningScheme().isReplicateNullsAndAny())) + .global(globalPartitioning) .constants(constants) .build(); + } case REPLICATE: // TODO: this should have the same global properties as the stream taking the replicated data return ActualProperties.builder() 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 4c65a9bf8d42..90510f8d111d 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 @@ -169,6 +169,7 @@ public PlanNode visitExchange(ExchangeNode node, RewriteContext case GATHER: return StreamProperties.singleStream(); case REPARTITION: - if (node.getPartitioningScheme().getPartitioning().getHandle().equals(FIXED_ARBITRARY_DISTRIBUTION)) { + if (node.getPartitioningScheme().getPartitioning().getHandle().equals(FIXED_ARBITRARY_DISTRIBUTION) || + // no strict partitioning guarantees when multiple writers per partitions are allows (scaled writers) + node.getPartitioningScheme().isScaleWriters()) { return new StreamProperties(FIXED, Optional.empty(), false); } checkArgument( diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/SymbolMapper.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/SymbolMapper.java index 2bab1de7f0ce..9ddb6f97662e 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/SymbolMapper.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/SymbolMapper.java @@ -304,6 +304,7 @@ private PartitioningScheme canonicalize(PartitioningScheme scheme, PlanNode sour mapAndDistinctVariable(source.getOutputVariables()), scheme.getHashColumn().map(this::map), scheme.isReplicateNullsAndAny(), + scheme.isScaleWriters(), scheme.getEncoding(), scheme.getBucketToPartition()); } 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 eff4ba73c396..9ff8ce55cfed 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 @@ -334,6 +334,7 @@ public PlanNode visitExchange(ExchangeNode node, RewriteContext context) outputs.build(), canonicalize(node.getPartitioningScheme().getHashColumn()), node.getPartitioningScheme().isReplicateNullsAndAny(), + node.getPartitioningScheme().isScaleWriters(), node.getPartitioningScheme().getEncoding(), node.getPartitioningScheme().getBucketToPartition()); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/ExchangeNode.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/ExchangeNode.java index 12928728b6d8..ed2b2307dccb 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/ExchangeNode.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/ExchangeNode.java @@ -192,6 +192,7 @@ public static ExchangeNode partitionedExchange(PlanNodeId id, Scope scope, PlanN child.getOutputVariables(), hashColumn, replicateNullsAndAny, + false, COLUMNAR, Optional.empty())); } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java b/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java index 0fd3a9f66209..120a48ca62fd 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java @@ -249,7 +249,8 @@ public void testDefaults() .setEagerPlanValidationEnabled(false) .setEagerPlanValidationThreadPoolSize(20) .setPrestoSparkExecutionEnvironment(false) - .setSingleNodeExecutionEnabled(false)); + .setSingleNodeExecutionEnabled(false) + .setNativeExecutionScaleWritersThreadsEnabled(false)); } @Test @@ -448,6 +449,7 @@ public void testExplicitPropertyMappings() .put("eager-plan-validation-thread-pool-size", "2") .put("presto-spark-execution-environment", "true") .put("single-node-execution-enabled", "true") + .put("native-execution-scale-writer-threads-enabled", "true") .build(); FeaturesConfig expected = new FeaturesConfig() @@ -643,7 +645,8 @@ public void testExplicitPropertyMappings() .setEagerPlanValidationEnabled(true) .setEagerPlanValidationThreadPoolSize(2) .setPrestoSparkExecutionEnvironment(true) - .setSingleNodeExecutionEnabled(true); + .setSingleNodeExecutionEnabled(true) + .setNativeExecutionScaleWritersThreadsEnabled(true); assertFullMapping(properties, expected); } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestCanonicalPlanGenerator.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestCanonicalPlanGenerator.java index ca678636d3ab..e38e5c4eae0c 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestCanonicalPlanGenerator.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestCanonicalPlanGenerator.java @@ -346,7 +346,7 @@ public void testCanonicalPartitioningScheme() .filter(f -> !f.isSynthetic()) .map(Field::getName) .collect(toImmutableSet()), - ImmutableSet.of("partitioning", "outputLayout", "hashColumn", "replicateNullsAndAny", "encoding", "bucketToPartition")); + ImmutableSet.of("partitioning", "outputLayout", "hashColumn", "replicateNullsAndAny", "scaleWriters", "encoding", "bucketToPartition")); assertEquals( Arrays.stream(Partitioning.class.getDeclaredFields()) .filter(f -> !f.isSynthetic()) diff --git a/presto-native-execution/presto_cpp/main/types/PrestoToVeloxQueryPlan.cpp b/presto-native-execution/presto_cpp/main/types/PrestoToVeloxQueryPlan.cpp index ce4e2fb32280..bdce271eb60c 100644 --- a/presto-native-execution/presto_cpp/main/types/PrestoToVeloxQueryPlan.cpp +++ b/presto-native-execution/presto_cpp/main/types/PrestoToVeloxQueryPlan.cpp @@ -345,6 +345,7 @@ VectorSerde::Kind toVeloxSerdeKind(protocol::ExchangeEncoding encoding) { std::shared_ptr buildLocalSystemPartitionNode( const std::shared_ptr& node, core::LocalPartitionNode::Type type, + bool scaleWriters, const RowTypePtr& outputType, std::vector&& sourceNodes, const VeloxExprConverter& exprConverter) { @@ -355,7 +356,7 @@ std::shared_ptr buildLocalSystemPartitionNode( return std::make_shared( node->id, type, - false, + scaleWriters, std::make_shared(outputType, keyChannels), std::move(sourceNodes)); } @@ -364,7 +365,7 @@ std::shared_ptr buildLocalSystemPartitionNode( return std::make_shared( node->id, type, - false, + scaleWriters, std::make_shared(), std::move(sourceNodes)); } @@ -433,12 +434,19 @@ core::PlanNodePtr VeloxQueryPlanConverterBase::toVeloxQueryPlan( return core::LocalPartitionNode::gather(node->id, std::move(sourceNodes)); } - auto connectorHandle = - node->partitioningScheme.partitioning.handle.connectorHandle; + const auto& partitioningScheme = node->partitioningScheme; + const auto& connectorHandle = + partitioningScheme.partitioning.handle.connectorHandle; + const bool scaleWriters = partitioningScheme.scaleWriters; if (std::dynamic_pointer_cast( connectorHandle) != nullptr) { return buildLocalSystemPartitionNode( - node, type, outputType, std::move(sourceNodes), exprConverter_); + node, + type, + scaleWriters, + outputType, + std::move(sourceNodes), + exprConverter_); } auto partitionKeys = toFieldExprs( @@ -459,7 +467,7 @@ core::PlanNodePtr VeloxQueryPlanConverterBase::toVeloxQueryPlan( return std::make_shared( node->id, type, - false, + scaleWriters, std::shared_ptr(std::move(spec)), std::move(sourceNodes)); } diff --git a/presto-native-execution/presto_cpp/main/types/tests/data/FinalAgg.json b/presto-native-execution/presto_cpp/main/types/tests/data/FinalAgg.json index 66ac079326a6..b401d3389b06 100644 --- a/presto-native-execution/presto_cpp/main/types/tests/data/FinalAgg.json +++ b/presto-native-execution/presto_cpp/main/types/tests/data/FinalAgg.json @@ -48,6 +48,7 @@ "type":"bigint" }, "replicateNullsAndAny":false, + "scaleWriters": false, "encoding":"COLUMNAR" }, "sources":[ @@ -241,6 +242,7 @@ } ], "replicateNullsAndAny":false, + "scaleWriters": false, "encoding":"COLUMNAR", "bucketToPartition":[ 0 diff --git a/presto-native-execution/presto_cpp/main/types/tests/data/OffsetLimit.json b/presto-native-execution/presto_cpp/main/types/tests/data/OffsetLimit.json index 517bffe31002..c85ab50621d4 100644 --- a/presto-native-execution/presto_cpp/main/types/tests/data/OffsetLimit.json +++ b/presto-native-execution/presto_cpp/main/types/tests/data/OffsetLimit.json @@ -52,6 +52,7 @@ } ], "replicateNullsAndAny":false, + "scaleWriters": false, "encoding":"COLUMNAR" }, "sources":[ @@ -104,6 +105,7 @@ } ], "replicateNullsAndAny":false, + "scaleWriters": false, "encoding":"COLUMNAR" }, "sources":[ @@ -156,6 +158,7 @@ } ], "replicateNullsAndAny":false, + "scaleWriters": false, "encoding":"COLUMNAR" }, "sources":[ @@ -481,6 +484,7 @@ ], "replicateNullsAndAny":false, "encoding":"COLUMNAR", + "scaleWriters": false, "bucketToPartition":[ 0 ] diff --git a/presto-native-execution/presto_cpp/main/types/tests/data/Output.json b/presto-native-execution/presto_cpp/main/types/tests/data/Output.json index 10f6452605c6..ba2b68cab6b8 100644 --- a/presto-native-execution/presto_cpp/main/types/tests/data/Output.json +++ b/presto-native-execution/presto_cpp/main/types/tests/data/Output.json @@ -90,6 +90,7 @@ } ], "replicateNullsAndAny":false, + "scaleWriters": false, "encoding":"COLUMNAR", "bucketToPartition":[ 0 diff --git a/presto-native-execution/presto_cpp/main/types/tests/data/PartitionedOutput.json b/presto-native-execution/presto_cpp/main/types/tests/data/PartitionedOutput.json index 19bc5f375622..5fb027dddb22 100644 --- a/presto-native-execution/presto_cpp/main/types/tests/data/PartitionedOutput.json +++ b/presto-native-execution/presto_cpp/main/types/tests/data/PartitionedOutput.json @@ -458,6 +458,7 @@ } ], "replicateNullsAndAny":false, + "scaleWriters": false, "bucketToPartition":[ 0, 1, diff --git a/presto-native-execution/presto_cpp/main/types/tests/data/ScanAgg.json b/presto-native-execution/presto_cpp/main/types/tests/data/ScanAgg.json index 20a54c15041b..cedb6f0483e2 100644 --- a/presto-native-execution/presto_cpp/main/types/tests/data/ScanAgg.json +++ b/presto-native-execution/presto_cpp/main/types/tests/data/ScanAgg.json @@ -320,6 +320,7 @@ } ], "replicateNullsAndAny":false, + "scaleWriters": false, "encoding":"COLUMNAR", "bucketToPartition":[ 0 diff --git a/presto-native-execution/presto_cpp/main/types/tests/data/ScanAggBatch.json b/presto-native-execution/presto_cpp/main/types/tests/data/ScanAggBatch.json index c7c257398ff3..d2a2c7d2f825 100644 --- a/presto-native-execution/presto_cpp/main/types/tests/data/ScanAggBatch.json +++ b/presto-native-execution/presto_cpp/main/types/tests/data/ScanAggBatch.json @@ -299,6 +299,7 @@ } ], "replicateNullsAndAny":false, + "scaleWriters": false, "encoding":"COLUMNAR", "bucketToPartition":[ 0, diff --git a/presto-native-execution/presto_cpp/main/types/tests/data/ScanAggCustomConnectorId.json b/presto-native-execution/presto_cpp/main/types/tests/data/ScanAggCustomConnectorId.json index d6f4496daae4..ea974c78d599 100644 --- a/presto-native-execution/presto_cpp/main/types/tests/data/ScanAggCustomConnectorId.json +++ b/presto-native-execution/presto_cpp/main/types/tests/data/ScanAggCustomConnectorId.json @@ -266,6 +266,7 @@ } ], "replicateNullsAndAny":false, + "scaleWriters": false, "encoding":"COLUMNAR", "bucketToPartition":[ 0 diff --git a/presto-native-execution/presto_cpp/main/types/tests/data/ValuesPipeTest.json b/presto-native-execution/presto_cpp/main/types/tests/data/ValuesPipeTest.json index c1d4e930f42d..da7c5a5f5194 100644 --- a/presto-native-execution/presto_cpp/main/types/tests/data/ValuesPipeTest.json +++ b/presto-native-execution/presto_cpp/main/types/tests/data/ValuesPipeTest.json @@ -35,6 +35,7 @@ } ], "replicateNullsAndAny": false, + "scaleWriters": false, "encoding":"COLUMNAR" }, "sources": [ @@ -203,6 +204,7 @@ } ], "replicateNullsAndAny": false, + "scaleWriters": false, "encoding":"COLUMNAR", "bucketToPartition": [ 0 diff --git a/presto-native-execution/presto_cpp/presto_protocol/core/presto_protocol_core.cpp b/presto-native-execution/presto_cpp/presto_protocol/core/presto_protocol_core.cpp index 3cfabe64627f..3dcff5034cdd 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/core/presto_protocol_core.cpp +++ b/presto-native-execution/presto_cpp/presto_protocol/core/presto_protocol_core.cpp @@ -4828,6 +4828,13 @@ void to_json(json& j, const PartitioningScheme& p) { "PartitioningScheme", "bool", "replicateNullsAndAny"); + to_json_key( + j, + "scaleWriters", + p.scaleWriters, + "PartitioningScheme", + "bool", + "scaleWriters"); to_json_key( j, "encoding", @@ -4873,6 +4880,13 @@ void from_json(const json& j, PartitioningScheme& p) { "PartitioningScheme", "bool", "replicateNullsAndAny"); + from_json_key( + j, + "scaleWriters", + p.scaleWriters, + "PartitioningScheme", + "bool", + "scaleWriters"); from_json_key( j, "encoding", diff --git a/presto-native-execution/presto_cpp/presto_protocol/core/presto_protocol_core.h b/presto-native-execution/presto_cpp/presto_protocol/core/presto_protocol_core.h index a50e7da4839e..f429324ac8d5 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/core/presto_protocol_core.h +++ b/presto-native-execution/presto_cpp/presto_protocol/core/presto_protocol_core.h @@ -1308,6 +1308,7 @@ struct PartitioningScheme { List outputLayout = {}; std::shared_ptr hashColumn = {}; bool replicateNullsAndAny = {}; + bool scaleWriters = {}; ExchangeEncoding encoding = {}; std::shared_ptr> bucketToPartition = {}; }; diff --git a/presto-native-execution/presto_cpp/presto_protocol/tests/data/ExchangeNode.json b/presto-native-execution/presto_cpp/presto_protocol/tests/data/ExchangeNode.json index cabec3d73bf9..693c5445bc2c 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/tests/data/ExchangeNode.json +++ b/presto-native-execution/presto_cpp/presto_protocol/tests/data/ExchangeNode.json @@ -27,6 +27,7 @@ } ], "replicateNullsAndAny": false, + "scaleWriters": true, "encoding":"COLUMNAR" }, "sources": [], diff --git a/presto-native-execution/presto_cpp/presto_protocol/tests/data/FilterNode.json b/presto-native-execution/presto_cpp/presto_protocol/tests/data/FilterNode.json index 7da48e2fc6be..da24ea42253a 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/tests/data/FilterNode.json +++ b/presto-native-execution/presto_cpp/presto_protocol/tests/data/FilterNode.json @@ -30,6 +30,7 @@ } ], "replicateNullsAndAny": false, + "scaleWriters": false, "encoding":"COLUMNAR" }, "sources": [], diff --git a/presto-native-execution/presto_cpp/presto_protocol/tests/data/OutputNode.json b/presto-native-execution/presto_cpp/presto_protocol/tests/data/OutputNode.json index b2811c1227e4..f8f76ba49056 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/tests/data/OutputNode.json +++ b/presto-native-execution/presto_cpp/presto_protocol/tests/data/OutputNode.json @@ -30,6 +30,7 @@ } ], "replicateNullsAndAny": false, + "scaleWriters": false, "encoding":"COLUMNAR" }, "sources": [], diff --git a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/TestPrestoNativeWriter.java b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/TestPrestoNativeWriter.java index 34fcd07a561e..3b12df96dd90 100644 --- a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/TestPrestoNativeWriter.java +++ b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/TestPrestoNativeWriter.java @@ -14,15 +14,31 @@ package com.facebook.presto.nativeworker; import com.facebook.presto.Session; +import com.facebook.presto.spi.plan.TableWriterNode; +import com.facebook.presto.sql.planner.Plan; +import com.facebook.presto.sql.planner.plan.ExchangeNode; import com.facebook.presto.testing.ExpectedQueryRunner; import com.facebook.presto.testing.QueryRunner; import com.facebook.presto.tests.AbstractTestQueryFramework; +import com.google.common.base.Joiner; import com.google.common.collect.ImmutableList; import org.testng.annotations.Test; +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.OptionalLong; import java.util.UUID; +import java.util.function.Consumer; +import static com.facebook.presto.SystemSessionProperties.NATIVE_EXECUTION_SCALE_WRITER_THREADS_ENABLED; +import static com.facebook.presto.SystemSessionProperties.SCALE_WRITERS; +import static com.facebook.presto.SystemSessionProperties.TASK_CONCURRENCY; +import static com.facebook.presto.SystemSessionProperties.TASK_PARTITIONED_WRITER_COUNT; +import static com.facebook.presto.SystemSessionProperties.TASK_WRITER_COUNT; import static com.facebook.presto.common.type.BigintType.BIGINT; +import static com.facebook.presto.hive.HiveSessionProperties.SHUFFLE_PARTITIONED_COLUMNS_FOR_TABLE_WRITE; import static com.facebook.presto.nativeworker.NativeQueryRunnerUtils.createBucketedCustomer; import static com.facebook.presto.nativeworker.NativeQueryRunnerUtils.createBucketedLineitemAndOrders; import static com.facebook.presto.nativeworker.NativeQueryRunnerUtils.createCustomer; @@ -36,9 +52,14 @@ import static com.facebook.presto.nativeworker.NativeQueryRunnerUtils.createPrestoBenchTables; import static com.facebook.presto.nativeworker.NativeQueryRunnerUtils.createRegion; import static com.facebook.presto.nativeworker.NativeQueryRunnerUtils.createSupplier; +import static com.facebook.presto.sql.planner.SystemPartitioningHandle.FIXED_ARBITRARY_DISTRIBUTION; +import static com.facebook.presto.sql.planner.SystemPartitioningHandle.SCALED_WRITER_DISTRIBUTION; +import static com.facebook.presto.sql.planner.optimizations.PlanNodeSearcher.searchFrom; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; public class TestPrestoNativeWriter extends AbstractTestQueryFramework @@ -303,7 +324,7 @@ public void testCreateBucketSortedTableAsSelect() } @Test - public void testScaleWriters() + public void testScaleWriterTasks() { Session session = buildSessionForTableWrite(); String tmpTableName = generateRandomTableName(); @@ -321,6 +342,228 @@ public void testScaleWriters() dropTableIfExists(tmpTableName); } + @Test + public void testScaleWriterThreads() + { + // no scaling for bucketed tables + testScaledWriters( + /*partitioned*/ true, + /*shufflePartitionedColumns*/ false, + /*bucketed*/ true, + /*scaleWriterTasks*/ true, + /*scaleWriterThreads*/ true, + plan -> { + TableWriterNode tableWriteNode = searchFrom(plan.getRoot()) + .where(node -> node instanceof TableWriterNode) + .findOnlyElement(); + ExchangeNode localExchange = (ExchangeNode) tableWriteNode.getSource(); + assertFalse(localExchange.getPartitioningScheme().isScaleWriters()); + ExchangeNode remoteExchange = (ExchangeNode) localExchange.getSources().get(0); + assertFalse(remoteExchange.getPartitioningScheme().isScaleWriters()); + }); + + // no scaling when shuffle_partitioned_columns_for_table_write is requested + testScaledWriters( + /*partitioned*/ true, + /*shufflePartitionedColumns*/ true, + /*bucketed*/ false, + /*scaleWriterTasks*/ true, + /*scaleWriterThreads*/ true, + plan -> { + TableWriterNode tableWriteNode = searchFrom(plan.getRoot()) + .where(node -> node instanceof TableWriterNode) + .findOnlyElement(); + ExchangeNode localExchange = (ExchangeNode) tableWriteNode.getSource(); + assertFalse(localExchange.getPartitioningScheme().isScaleWriters()); + ExchangeNode remoteExchange = (ExchangeNode) localExchange.getSources().get(0); + assertFalse(remoteExchange.getPartitioningScheme().isScaleWriters()); + }); + + // scale tasks and threads for partitioned tables if enabled + testScaledWriters( + /*partitioned*/ true, + /*shufflePartitionedColumns*/ false, + /*bucketed*/ false, + /*scaleWriterTasks*/ true, + /*scaleWriterThreads*/ true, + plan -> { + TableWriterNode tableWriteNode = searchFrom(plan.getRoot()) + .where(node -> node instanceof TableWriterNode) + .findOnlyElement(); + ExchangeNode localExchange = (ExchangeNode) tableWriteNode.getSource(); + assertTrue(localExchange.getPartitioningScheme().isScaleWriters()); + assertThat(localExchange.getPartitioningScheme().getPartitioning().getArguments()) + // single partitioning column + .hasSize(1); + assertThat(localExchange.getPartitioningScheme().getPartitioning().getHandle().getConnectorId()) + .isPresent(); + ExchangeNode remoteExchange = (ExchangeNode) localExchange.getSources().get(0); + assertEquals(remoteExchange.getPartitioningScheme().getPartitioning().getHandle(), SCALED_WRITER_DISTRIBUTION); + }); + testScaledWriters( + /*partitioned*/ true, + /*shufflePartitionedColumns*/ false, + /*bucketed*/ false, + /*scaleWriterTasks*/ true, + /*scaleWriterThreads*/ false, + plan -> { + TableWriterNode tableWriteNode = searchFrom(plan.getRoot()) + .where(node -> node instanceof TableWriterNode) + .findOnlyElement(); + ExchangeNode localExchange = (ExchangeNode) tableWriteNode.getSource(); + assertFalse(localExchange.getPartitioningScheme().isScaleWriters()); + assertEquals(localExchange.getPartitioningScheme().getPartitioning().getHandle(), FIXED_ARBITRARY_DISTRIBUTION); + ExchangeNode remoteExchange = (ExchangeNode) localExchange.getSources().get(0); + assertEquals(remoteExchange.getPartitioningScheme().getPartitioning().getHandle(), SCALED_WRITER_DISTRIBUTION); + }); + testScaledWriters( + /*partitioned*/ true, + /*shufflePartitionedColumns*/ false, + /*bucketed*/ false, + /*scaleWriterTasks*/ false, + /*scaleWriterThreads*/ true, + plan -> { + TableWriterNode tableWriteNode = searchFrom(plan.getRoot()) + .where(node -> node instanceof TableWriterNode) + .findOnlyElement(); + ExchangeNode localExchange = (ExchangeNode) tableWriteNode.getSource(); + assertTrue(localExchange.getPartitioningScheme().isScaleWriters()); + assertThat(localExchange.getPartitioningScheme().getPartitioning().getArguments()) + // single partitioning column + .hasSize(1); + }); + testScaledWriters( + /*partitioned*/ true, + /*shufflePartitionedColumns*/ false, + /*bucketed*/ false, + /*scaleWriterTasks*/ false, + /*scaleWriterThreads*/ false, + plan -> { + TableWriterNode tableWriteNode = searchFrom(plan.getRoot()) + .where(node -> node instanceof TableWriterNode) + .findOnlyElement(); + ExchangeNode localExchange = (ExchangeNode) tableWriteNode.getSource(); + assertFalse(localExchange.getPartitioningScheme().isScaleWriters()); + assertEquals(localExchange.getPartitioningScheme().getPartitioning().getHandle(), FIXED_ARBITRARY_DISTRIBUTION); + }); + + // scale thread writers for non partitioned tables + testScaledWriters( + /*partitioned*/ false, + /*shufflePartitionedColumns*/ false, + /*bucketed*/ false, + /*scaleWriterTasks*/ true, + /*scaleWriterThreads*/ true, + plan -> { + TableWriterNode tableWriteNode = searchFrom(plan.getRoot()) + .where(node -> node instanceof TableWriterNode) + .findOnlyElement(); + ExchangeNode localExchange = (ExchangeNode) tableWriteNode.getSource(); + assertTrue(localExchange.getPartitioningScheme().isScaleWriters()); + assertEquals(localExchange.getPartitioningScheme().getPartitioning().getHandle(), FIXED_ARBITRARY_DISTRIBUTION); + ExchangeNode remoteExchange = (ExchangeNode) localExchange.getSources().get(0); + assertEquals(remoteExchange.getPartitioningScheme().getPartitioning().getHandle(), SCALED_WRITER_DISTRIBUTION); + }); + } + + private void testScaledWriters( + boolean partitioned, + boolean shufflePartitionedColumns, + boolean bucketed, + boolean scaleWriterTasks, + boolean scaleWriterThreads, + Consumer planAssertion) + { + String tableName = generateRandomTableName(); + OptionalLong bucketCount = OptionalLong.empty(); + long partitionCount = 1; + + Map columns = new LinkedHashMap<>(); + columns.put("orderkey", "BIGINT"); + columns.put("custkey", "BIGINT"); + columns.put("orderstatus", "VARCHAR"); + + List properties = new ArrayList<>(); + properties.add("format = 'DWRF'"); + if (partitioned) { + properties.add("partitioned_by = ARRAY['orderstatus']"); + partitionCount = 3; + } + if (bucketed) { + properties.add("bucketed_by = ARRAY['orderkey']"); + bucketCount = OptionalLong.of(3); + properties.add("bucket_count = " + bucketCount.getAsLong()); + } + + Session session = Session.builder(getSession()) + .setSystemProperty(SCALE_WRITERS, scaleWriterTasks + "") + .setSystemProperty(NATIVE_EXECUTION_SCALE_WRITER_THREADS_ENABLED, scaleWriterThreads + "") + .setSystemProperty(TASK_WRITER_COUNT, "4") + .setSystemProperty(TASK_PARTITIONED_WRITER_COUNT, "8") + .setSystemProperty(TASK_CONCURRENCY, "16") + .setCatalogSessionProperty("hive", SHUFFLE_PARTITIONED_COLUMNS_FOR_TABLE_WRITE, shufflePartitionedColumns + "") + .build(); + + String createTable = format( + "CREATE TABLE %s (%s) WITH (%s)", + tableName, + Joiner.on(", ") + .withKeyValueSeparator(" ") + .join(columns), + Joiner.on(", ").join(properties)); + assertUpdate(createTable); + + long numberOfRowsInOrdersTable = (long) getQueryRunner().execute("SELECT count(*) FROM orders").getOnlyValue(); + + String insert = format( + "INSERT INTO %s SELECT %s FROM orders", + tableName, + Joiner.on(", ").join(columns.keySet())); + assertUpdate(session, insert, numberOfRowsInOrdersTable, planAssertion); + assertQuery( + format( + "SELECT %s FROM %s", + Joiner.on(", ").join(columns.keySet()), + tableName), + format( + "SELECT %s FROM orders", + Joiner.on(", ").join(columns.keySet()))); + assertFileCount(tableName, partitionCount, bucketCount); + + assertUpdate(format("DROP TABLE %s", tableName)); + + tableName = generateRandomTableName(); + + String ctas = format( + "CREATE TABLE %s WITH (%s) AS SELECT %s FROM orders", + tableName, + Joiner.on(", ").join(properties), + Joiner.on(", ").join(columns.keySet())); + assertUpdate(session, ctas, numberOfRowsInOrdersTable, planAssertion); + assertQuery( + format( + "SELECT %s FROM %s", + Joiner.on(", ").join(columns.keySet()), + tableName), + format( + "SELECT %s FROM orders", + Joiner.on(", ").join(columns.keySet()))); + assertFileCount(tableName, partitionCount, bucketCount); + + assertUpdate(format("DROP TABLE %s", tableName)); + } + + private void assertFileCount(String tableName, long partitionCount, OptionalLong bucketCount) + { + long actual = (long) computeActual(format("SELECT count(DISTINCT \"$path\") FROM %s", tableName)).getOnlyValue(); + if (bucketCount.isPresent()) { + assertEquals(actual, bucketCount.getAsLong() * partitionCount); + } + else { + assertThat(actual).isGreaterThanOrEqualTo(partitionCount); + } + } + @Test public void testCollectColumnStatisticsOnCreateTable() { diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/ConnectorNewTableLayout.java b/presto-spi/src/main/java/com/facebook/presto/spi/ConnectorNewTableLayout.java index ea4cbecefb5c..46fcf1eb665f 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/ConnectorNewTableLayout.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/ConnectorNewTableLayout.java @@ -18,17 +18,25 @@ import java.util.List; import java.util.Objects; +import static com.facebook.presto.spi.PartitionedTableWritePolicy.SINGLE_WRITER_PER_PARTITION_REQUIRED; import static java.util.Objects.requireNonNull; public class ConnectorNewTableLayout { private final ConnectorPartitioningHandle partitioning; private final List partitionColumns; + private final PartitionedTableWritePolicy writerPolicy; public ConnectorNewTableLayout(ConnectorPartitioningHandle partitioning, List partitionColumns) + { + this(partitioning, partitionColumns, SINGLE_WRITER_PER_PARTITION_REQUIRED); + } + + public ConnectorNewTableLayout(ConnectorPartitioningHandle partitioning, List partitionColumns, PartitionedTableWritePolicy writerPolicy) { this.partitioning = requireNonNull(partitioning, "partitioning is null"); this.partitionColumns = requireNonNull(partitionColumns, "partitionColumns is null"); + this.writerPolicy = requireNonNull(writerPolicy, "writerPolicy is null"); } public ConnectorPartitioningHandle getPartitioning() @@ -41,6 +49,11 @@ public List getPartitionColumns() return partitionColumns; } + public PartitionedTableWritePolicy getWriterPolicy() + { + return writerPolicy; + } + @Override public boolean equals(Object o) { @@ -52,12 +65,13 @@ public boolean equals(Object o) } ConnectorNewTableLayout that = (ConnectorNewTableLayout) o; return Objects.equals(partitioning, that.partitioning) && - Objects.equals(partitionColumns, that.partitionColumns); + Objects.equals(partitionColumns, that.partitionColumns) && + Objects.equals(writerPolicy, that.writerPolicy); } @Override public int hashCode() { - return Objects.hash(partitioning, partitionColumns); + return Objects.hash(partitioning, partitionColumns, writerPolicy); } } diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/NewTableLayout.java b/presto-spi/src/main/java/com/facebook/presto/spi/NewTableLayout.java index bd9d743fe6a4..202b25c9a492 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/NewTableLayout.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/NewTableLayout.java @@ -64,6 +64,11 @@ public List getPartitionColumns() return layout.getPartitionColumns(); } + public PartitionedTableWritePolicy getWriterPolicy() + { + return layout.getWriterPolicy(); + } + @Override public boolean equals(Object o) { diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/PartitionedTableWritePolicy.java b/presto-spi/src/main/java/com/facebook/presto/spi/PartitionedTableWritePolicy.java new file mode 100644 index 000000000000..9d49184ea81e --- /dev/null +++ b/presto-spi/src/main/java/com/facebook/presto/spi/PartitionedTableWritePolicy.java @@ -0,0 +1,20 @@ +/* + * 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; + +public enum PartitionedTableWritePolicy +{ + SINGLE_WRITER_PER_PARTITION_REQUIRED, + MULTIPLE_WRITERS_PER_PARTITION_ALLOWED, +} diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/plan/PartitioningScheme.java b/presto-spi/src/main/java/com/facebook/presto/spi/plan/PartitioningScheme.java index 244772fd50b8..0830fd02b6cb 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/plan/PartitioningScheme.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/plan/PartitioningScheme.java @@ -37,9 +37,22 @@ public class PartitioningScheme private final List outputLayout; private final Optional hashColumn; private final boolean replicateNullsAndAny; + private final boolean scaleWriters; private final ExchangeEncoding encoding; private final Optional bucketToPartition; + public PartitioningScheme(Partitioning partitioning, List outputLayout, boolean scaleWriters) + { + this( + partitioning, + outputLayout, + Optional.empty(), + false, + scaleWriters, + COLUMNAR, + Optional.empty()); + } + public PartitioningScheme(Partitioning partitioning, List outputLayout) { this( @@ -47,6 +60,7 @@ public PartitioningScheme(Partitioning partitioning, List outputLayout, @JsonProperty("hashColumn") Optional hashColumn, @JsonProperty("replicateNullsAndAny") boolean replicateNullsAndAny, + @JsonProperty("scaleWriters") boolean scaleWriters, @JsonProperty("encoding") ExchangeEncoding encoding, @JsonProperty("bucketToPartition") Optional bucketToPartition) { @@ -85,6 +101,7 @@ public PartitioningScheme( checkArgument(!replicateNullsAndAny || columns.size() <= 1, "Must have at most one partitioning column when nullPartition is REPLICATE."); this.replicateNullsAndAny = replicateNullsAndAny; + this.scaleWriters = scaleWriters; this.encoding = requireNonNull(encoding, "encoding is null"); this.bucketToPartition = requireNonNull(bucketToPartition, "bucketToPartition is null"); } @@ -119,6 +136,12 @@ public ExchangeEncoding getEncoding() return encoding; } + @JsonProperty + public boolean isScaleWriters() + { + return scaleWriters; + } + @JsonProperty public Optional getBucketToPartition() { @@ -127,12 +150,26 @@ public Optional getBucketToPartition() public PartitioningScheme withBucketToPartition(Optional bucketToPartition) { - return new PartitioningScheme(partitioning, outputLayout, hashColumn, replicateNullsAndAny, encoding, bucketToPartition); + return new PartitioningScheme( + partitioning, + outputLayout, + hashColumn, + replicateNullsAndAny, + scaleWriters, + encoding, + bucketToPartition); } public PartitioningScheme withRowWiseEncoding() { - return new PartitioningScheme(partitioning, outputLayout, hashColumn, replicateNullsAndAny, ROW_WISE, bucketToPartition); + return new PartitioningScheme( + partitioning, + outputLayout, + hashColumn, + replicateNullsAndAny, + scaleWriters, + ROW_WISE, + bucketToPartition); } public boolean isSingleOrBroadcastOrArbitrary() @@ -153,6 +190,7 @@ public boolean equals(Object o) return Objects.equals(partitioning, that.partitioning) && Objects.equals(outputLayout, that.outputLayout) && replicateNullsAndAny == that.replicateNullsAndAny && + scaleWriters == that.scaleWriters && encoding == that.encoding && Objects.equals(bucketToPartition, that.bucketToPartition); } @@ -160,7 +198,7 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash(partitioning, outputLayout, replicateNullsAndAny, encoding, bucketToPartition); + return Objects.hash(partitioning, outputLayout, replicateNullsAndAny, encoding, scaleWriters, bucketToPartition); } @Override @@ -170,6 +208,7 @@ public String toString() ", outputLayout=" + outputLayout + ", hashChannel=" + hashColumn + ", replicateNullsAndAny=" + replicateNullsAndAny + + ", scaleWriters=" + scaleWriters + ", encoding=" + encoding + ", bucketToPartition=" + bucketToPartition + '}'; diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/plan/TableWriterNode.java b/presto-spi/src/main/java/com/facebook/presto/spi/plan/TableWriterNode.java index a693de9334cc..579727c61c2f 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/plan/TableWriterNode.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/plan/TableWriterNode.java @@ -269,6 +269,11 @@ public PlanNode assignStatsEquivalentPlanNode(Optional statsEquivalent taskCountIfScaledWriter, isTemporaryTableWriter); } + public boolean isSingleWriterPerPartitionRequired() + { + return tablePartitioningScheme.isPresent() && !tablePartitioningScheme.get().isScaleWriters(); + } + // only used during planning -- will not be serialized @SuppressWarnings({"EmptyClass", "ClassMayBeInterface"}) public abstract static class WriterTarget From 737f5e17681a8cac07702901171e44406ccfc123 Mon Sep 17 00:00:00 2001 From: Jiaqi Zhang Date: Tue, 17 Dec 2024 11:14:15 -0800 Subject: [PATCH 20/25] Deprecate regex property for tracing --- .../src/main/sphinx/presto_cpp/properties-session.rst | 9 --------- .../NativeWorkerSessionPropertyProvider.java | 5 ----- .../presto_cpp/main/SessionProperties.cpp | 8 -------- .../presto_cpp/main/SessionProperties.h | 5 ----- 4 files changed, 27 deletions(-) diff --git a/presto-docs/src/main/sphinx/presto_cpp/properties-session.rst b/presto-docs/src/main/sphinx/presto_cpp/properties-session.rst index 47977a0627d2..6aa098e11bcd 100644 --- a/presto-docs/src/main/sphinx/presto_cpp/properties-session.rst +++ b/presto-docs/src/main/sphinx/presto_cpp/properties-session.rst @@ -378,15 +378,6 @@ The fragment id to be traced. If not specified, all fragments will be matched. The shard id to be traced. If not specified, all shards will be matched. -``native_query_trace_task_reg_exp`` -^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ - -* **Type:** ``varchar`` -* **Default value:** ``""`` - -The regular expression to match a task for tracing. It will be deprecated if there is -no issue with native_query_trace_fragment_id and native_query_trace_shard_id. - ``native_scaled_writer_rebalance_max_memory_usage_ratio`` ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ diff --git a/presto-main/src/main/java/com/facebook/presto/sessionpropertyproviders/NativeWorkerSessionPropertyProvider.java b/presto-main/src/main/java/com/facebook/presto/sessionpropertyproviders/NativeWorkerSessionPropertyProvider.java index caf3de920c3f..126a79382782 100644 --- a/presto-main/src/main/java/com/facebook/presto/sessionpropertyproviders/NativeWorkerSessionPropertyProvider.java +++ b/presto-main/src/main/java/com/facebook/presto/sessionpropertyproviders/NativeWorkerSessionPropertyProvider.java @@ -62,7 +62,6 @@ public class NativeWorkerSessionPropertyProvider public static final String NATIVE_QUERY_TRACE_DIR = "native_query_trace_dir"; public static final String NATIVE_QUERY_TRACE_NODE_IDS = "native_query_trace_node_ids"; public static final String NATIVE_QUERY_TRACE_MAX_BYTES = "native_query_trace_max_bytes"; - public static final String NATIVE_QUERY_TRACE_REG_EXP = "native_query_trace_task_reg_exp"; public static final String NATIVE_QUERY_TRACE_FRAGMENT_ID = "native_query_trace_fragment_id"; public static final String NATIVE_QUERY_TRACE_SHARD_ID = "native_query_trace_shard_id"; public static final String NATIVE_MAX_LOCAL_EXCHANGE_PARTITION_COUNT = "native_max_local_exchange_partition_count"; @@ -237,10 +236,6 @@ public NativeWorkerSessionPropertyProvider(FeaturesConfig featuresConfig) "The max trace bytes limit. Tracing is disabled if zero.", 0L, !nativeExecution), - stringProperty(NATIVE_QUERY_TRACE_REG_EXP, - "The regexp of traced task id. We only enable trace on a task if its id matches.", - "", - !nativeExecution), stringProperty(NATIVE_OP_TRACE_DIR_CREATE_CONFIG, "Config used to create operator trace directory. This config is provided to underlying file system and the config is free form. The form should be defined by the underlying file system.", "", diff --git a/presto-native-execution/presto_cpp/main/SessionProperties.cpp b/presto-native-execution/presto_cpp/main/SessionProperties.cpp index 314e81c3110f..fab89fda0bf7 100644 --- a/presto-native-execution/presto_cpp/main/SessionProperties.cpp +++ b/presto-native-execution/presto_cpp/main/SessionProperties.cpp @@ -306,14 +306,6 @@ SessionProperties::SessionProperties() { QueryConfig::kQueryTraceMaxBytes, std::to_string(c.queryTraceMaxBytes())); - addSessionProperty( - kQueryTraceTaskRegExp, - "The regexp of traced task id. We only enable trace on a task if its id" - " matches.", - VARCHAR(), - false, - QueryConfig::kQueryTraceTaskRegExp, - c.queryTraceTaskRegExp()); addSessionProperty( kOpTraceDirectoryCreateConfig, diff --git a/presto-native-execution/presto_cpp/main/SessionProperties.h b/presto-native-execution/presto_cpp/main/SessionProperties.h index a952c052845b..8966892e025c 100644 --- a/presto-native-execution/presto_cpp/main/SessionProperties.h +++ b/presto-native-execution/presto_cpp/main/SessionProperties.h @@ -232,11 +232,6 @@ class SessionProperties { static constexpr const char* kQueryTraceMaxBytes = "native_query_trace_max_bytes"; - /// The regexp of traced task id. We only enable trace on a task if its id - /// matches. - static constexpr const char* kQueryTraceTaskRegExp = - "native_query_trace_task_reg_exp"; - /// Config used to create operator trace directory. This config is provided to /// underlying file system and the config is free form. The form should be /// defined by the underlying file system. From eae4696aa220cf5297bc711b85661ded5c75f0c9 Mon Sep 17 00:00:00 2001 From: wangd Date: Sat, 21 Dec 2024 01:21:42 +0800 Subject: [PATCH 21/25] [Iceberg]Fix flaky caused by rename view test --- .../iceberg/TestIcebergMetadataListing.java | 22 ++++++++++--------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergMetadataListing.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergMetadataListing.java index 4b18df2cacce..0c7cb8e8b324 100644 --- a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergMetadataListing.java +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergMetadataListing.java @@ -157,16 +157,18 @@ public void testTableValidation() @Test public void testRenameView() { - assertQuerySucceeds("CREATE TABLE iceberg.test_schema.iceberg_test_table (_string VARCHAR, _integer INTEGER)"); - assertUpdate("CREATE VIEW iceberg.test_schema.test_view_to_be_renamed AS SELECT * FROM iceberg.test_schema.iceberg_test_table"); - assertUpdate("ALTER VIEW IF EXISTS iceberg.test_schema.test_view_to_be_renamed RENAME TO iceberg.test_schema.test_view_renamed"); - assertUpdate("CREATE VIEW iceberg.test_schema.test_view2_to_be_renamed AS SELECT * FROM iceberg.test_schema.iceberg_test_table"); - assertUpdate("ALTER VIEW iceberg.test_schema.test_view2_to_be_renamed RENAME TO iceberg.test_schema.test_view2_renamed"); - assertQuerySucceeds("SELECT * FROM iceberg.test_schema.test_view_renamed"); - assertQuerySucceeds("SELECT * FROM iceberg.test_schema.test_view2_renamed"); - assertUpdate("DROP VIEW iceberg.test_schema.test_view_renamed"); - assertUpdate("DROP VIEW iceberg.test_schema.test_view2_renamed"); - assertUpdate("DROP TABLE iceberg.test_schema.iceberg_test_table"); + assertQuerySucceeds("CREATE SCHEMA iceberg.test_rename_view_schema"); + assertQuerySucceeds("CREATE TABLE iceberg.test_rename_view_schema.iceberg_test_table (_string VARCHAR, _integer INTEGER)"); + assertUpdate("CREATE VIEW iceberg.test_rename_view_schema.test_view_to_be_renamed AS SELECT * FROM iceberg.test_rename_view_schema.iceberg_test_table"); + assertUpdate("ALTER VIEW IF EXISTS iceberg.test_rename_view_schema.test_view_to_be_renamed RENAME TO iceberg.test_rename_view_schema.test_view_renamed"); + assertUpdate("CREATE VIEW iceberg.test_rename_view_schema.test_view2_to_be_renamed AS SELECT * FROM iceberg.test_rename_view_schema.iceberg_test_table"); + assertUpdate("ALTER VIEW iceberg.test_rename_view_schema.test_view2_to_be_renamed RENAME TO iceberg.test_rename_view_schema.test_view2_renamed"); + assertQuerySucceeds("SELECT * FROM iceberg.test_rename_view_schema.test_view_renamed"); + assertQuerySucceeds("SELECT * FROM iceberg.test_rename_view_schema.test_view2_renamed"); + assertUpdate("DROP VIEW iceberg.test_rename_view_schema.test_view_renamed"); + assertUpdate("DROP VIEW iceberg.test_rename_view_schema.test_view2_renamed"); + assertUpdate("DROP TABLE iceberg.test_rename_view_schema.iceberg_test_table"); + assertQuerySucceeds("DROP SCHEMA IF EXISTS iceberg.test_rename_view_schema"); } @Test public void testRenameViewIfNotExists() From 2fcbb3fd76064a9245ee569bc37ce1dbb01b8c9d Mon Sep 17 00:00:00 2001 From: Xiaoxuan Meng Date: Fri, 20 Dec 2024 09:41:20 -0800 Subject: [PATCH 22/25] [native] Fix unsafe row exchange source with compression support --- .../operators/UnsafeRowExchangeSource.cpp | 22 ++++++++++++++----- .../main/operators/tests/BroadcastTest.cpp | 3 ++- .../presto_cpp/main/tests/TaskManagerTest.cpp | 3 ++- presto-native-execution/velox | 2 +- 4 files changed, 22 insertions(+), 8 deletions(-) diff --git a/presto-native-execution/presto_cpp/main/operators/UnsafeRowExchangeSource.cpp b/presto-native-execution/presto_cpp/main/operators/UnsafeRowExchangeSource.cpp index e952a62970e4..1c1453d393c2 100644 --- a/presto-native-execution/presto_cpp/main/operators/UnsafeRowExchangeSource.cpp +++ b/presto-native-execution/presto_cpp/main/operators/UnsafeRowExchangeSource.cpp @@ -16,6 +16,7 @@ #include "presto_cpp/main/common/Configs.h" #include "presto_cpp/main/operators/UnsafeRowExchangeSource.h" +#include "velox/serializers/RowSerializer.h" namespace facebook::presto::operators { @@ -36,7 +37,7 @@ UnsafeRowExchangeSource::request( return std::move(shuffle_->next()) .deferValue([this](velox::BufferPtr buffer) { std::vector promises; - int64_t totalBytes = 0; + int64_t totalBytes{0}; { std::lock_guard l(queue_->mutex()); @@ -45,14 +46,25 @@ UnsafeRowExchangeSource::request( queue_->enqueueLocked(nullptr, promises); } else { totalBytes = buffer->size(); + VELOX_CHECK_LE(totalBytes, std::numeric_limits::max()); ++numBatches_; - - auto ioBuf = - folly::IOBuf::wrapBuffer(buffer->as(), buffer->size()); + velox::serializer::detail::RowGroupHeader rowHeader{ + .uncompressedSize = static_cast(totalBytes), + .compressedSize = static_cast(totalBytes), + .compressed = false}; + auto headBuffer = std::make_shared( + velox::serializer::detail::RowGroupHeader::size(), '0'); + rowHeader.write(const_cast(headBuffer->data())); + + auto ioBuf = folly::IOBuf::wrapBuffer( + headBuffer->data(), headBuffer->size()); + ioBuf->appendToChain( + folly::IOBuf::wrapBuffer(buffer->as(), buffer->size())); queue_->enqueueLocked( std::make_unique( - std::move(ioBuf), [buffer](auto& /*unused*/) {}), + std::move(ioBuf), + [buffer, headBuffer](auto& /*unused*/) {}), promises); } } diff --git a/presto-native-execution/presto_cpp/main/operators/tests/BroadcastTest.cpp b/presto-native-execution/presto_cpp/main/operators/tests/BroadcastTest.cpp index 0973a8359c0b..f206ac6e1c71 100644 --- a/presto-native-execution/presto_cpp/main/operators/tests/BroadcastTest.cpp +++ b/presto-native-execution/presto_cpp/main/operators/tests/BroadcastTest.cpp @@ -216,7 +216,8 @@ class BroadcastTest : public exec::test::OperatorTestBase { pool(), dataType, velox::getNamedVectorSerde(velox::VectorSerde::Kind::kPresto), - &result); + &result, + nullptr); return result; } }; diff --git a/presto-native-execution/presto_cpp/main/tests/TaskManagerTest.cpp b/presto-native-execution/presto_cpp/main/tests/TaskManagerTest.cpp index 43190f2e5c03..8d92af63c56b 100644 --- a/presto-native-execution/presto_cpp/main/tests/TaskManagerTest.cpp +++ b/presto-native-execution/presto_cpp/main/tests/TaskManagerTest.cpp @@ -165,7 +165,8 @@ class Cursor { std::vector vectors; while (!input->atEnd()) { RowVectorPtr vector; - VectorStreamGroup::read(input.get(), pool_, rowType_, serde, &vector); + VectorStreamGroup::read( + input.get(), pool_, rowType_, serde, &vector, nullptr); vectors.emplace_back(vector); } return vectors; diff --git a/presto-native-execution/velox b/presto-native-execution/velox index 12942c1eb76d..9265fbfd9a07 160000 --- a/presto-native-execution/velox +++ b/presto-native-execution/velox @@ -1 +1 @@ -Subproject commit 12942c1eb76de019b775f4b207bc4595d8ace5c0 +Subproject commit 9265fbfd9a0716136456ef8b6e455ff110a0f7da From 1a7fd4efecde227d5cd3142c2b677fddb051fc2a Mon Sep 17 00:00:00 2001 From: Amit Dutta Date: Sat, 21 Dec 2024 14:54:17 -0800 Subject: [PATCH 23/25] [native] Advance velox. --- presto-native-execution/velox | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/presto-native-execution/velox b/presto-native-execution/velox index 9265fbfd9a07..e9bb6c18df11 160000 --- a/presto-native-execution/velox +++ b/presto-native-execution/velox @@ -1 +1 @@ -Subproject commit 9265fbfd9a0716136456ef8b6e455ff110a0f7da +Subproject commit e9bb6c18df1144e5581b1dc2ef71b210cae97f54 From 472fe92d3f7d79eb47b2070ccae7cbd0944a0f23 Mon Sep 17 00:00:00 2001 From: Xiaoxuan Meng Date: Sun, 22 Dec 2024 10:29:25 -0800 Subject: [PATCH 24/25] [native] Add support shuffle compression for Prestissimo --- .../presto_cpp/main/PrestoServer.cpp | 1 + .../presto_cpp/main/QueryContextManager.cpp | 11 +++++++++++ .../presto_cpp/main/SessionProperties.h | 4 ++++ 3 files changed, 16 insertions(+) diff --git a/presto-native-execution/presto_cpp/main/PrestoServer.cpp b/presto-native-execution/presto_cpp/main/PrestoServer.cpp index 84b3b26fde0a..1f316144bbfc 100644 --- a/presto-native-execution/presto_cpp/main/PrestoServer.cpp +++ b/presto-native-execution/presto_cpp/main/PrestoServer.cpp @@ -277,6 +277,7 @@ void PrestoServer::run() { registerPrestoToVeloxConnector( std::make_unique("$system@system")); + velox::exec::OutputBufferManager::initialize({}); initializeVeloxMemory(); initializeThreadPools(); diff --git a/presto-native-execution/presto_cpp/main/QueryContextManager.cpp b/presto-native-execution/presto_cpp/main/QueryContextManager.cpp index e12cdbaec3f5..9797fa5205f2 100644 --- a/presto-native-execution/presto_cpp/main/QueryContextManager.cpp +++ b/presto-native-execution/presto_cpp/main/QueryContextManager.cpp @@ -224,6 +224,17 @@ QueryContextManager::toVeloxConfigs( traceFragmentId = it.second; } else if (it.first == SessionProperties::kQueryTraceShardId) { traceShardId = it.second; + } else if (it.first == SessionProperties::kShuffleCompressionEnabled) { + if (it.second == "true") { + // NOTE: Presto java only support lz4 compression so configure the same + // compression kind on velox. + configs[core::QueryConfig::kShuffleCompressionKind] = std::to_string( + static_cast(velox::common::CompressionKind_LZ4)); + } else { + VELOX_USER_CHECK_EQ(it.second, "false"); + configs[core::QueryConfig::kShuffleCompressionKind] = std::to_string( + static_cast(velox::common::CompressionKind_NONE)); + } } else { configs[sessionProperties_.toVeloxConfig(it.first)] = it.second; sessionProperties_.updateVeloxConfig(it.first, it.second); diff --git a/presto-native-execution/presto_cpp/main/SessionProperties.h b/presto-native-execution/presto_cpp/main/SessionProperties.h index 8966892e025c..6d77bb8026f3 100644 --- a/presto-native-execution/presto_cpp/main/SessionProperties.h +++ b/presto-native-execution/presto_cpp/main/SessionProperties.h @@ -283,6 +283,10 @@ class SessionProperties { static constexpr const char* kPrefixSortMinRows = "native_prefixsort_min_rows"; + /// If true, enable the shuffle compression. + static constexpr const char* kShuffleCompressionEnabled = + "exchange_compression"; + SessionProperties(); const std::unordered_map>& From dba4fcaa2637f580addd9786505a72bf86df14d1 Mon Sep 17 00:00:00 2001 From: Amit Dutta Date: Mon, 23 Dec 2024 08:29:46 -0800 Subject: [PATCH 25/25] [native] Advance velox. --- presto-native-execution/velox | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/presto-native-execution/velox b/presto-native-execution/velox index e9bb6c18df11..8ebd3a8bae73 160000 --- a/presto-native-execution/velox +++ b/presto-native-execution/velox @@ -1 +1 @@ -Subproject commit e9bb6c18df1144e5581b1dc2ef71b210cae97f54 +Subproject commit 8ebd3a8bae73388485d13b99d7e6238b08c78561