diff --git a/pom.xml b/pom.xml index b41cfd7e6c754..8817698d8db6f 100644 --- a/pom.xml +++ b/pom.xml @@ -119,6 +119,8 @@ presto-tests presto-product-tests presto-jdbc + presto-pinot + presto-pinot-toolkit presto-cli presto-benchmark-driver presto-server @@ -293,6 +295,18 @@ ${project.version} + + com.facebook.presto + presto-pinot + ${project.version} + + + + com.facebook.presto + presto-pinot-toolkit + ${project.version} + + com.facebook.presto presto-mysql @@ -1235,6 +1249,12 @@ 3.1.4-1 + + com.facebook.presto.pinot + pinot-driver + 0.1.1 + + org.javassist diff --git a/presto-base-jdbc/src/main/java/com/facebook/presto/plugin/jdbc/optimization/JdbcPlanOptimizerProvider.java b/presto-base-jdbc/src/main/java/com/facebook/presto/plugin/jdbc/optimization/JdbcPlanOptimizerProvider.java index 94042314c7922..9dcd399b66472 100644 --- a/presto-base-jdbc/src/main/java/com/facebook/presto/plugin/jdbc/optimization/JdbcPlanOptimizerProvider.java +++ b/presto-base-jdbc/src/main/java/com/facebook/presto/plugin/jdbc/optimization/JdbcPlanOptimizerProvider.java @@ -53,7 +53,13 @@ public JdbcPlanOptimizerProvider( } @Override - public Set getConnectorPlanOptimizers() + public Set getLogicalPlanOptimizers() + { + return ImmutableSet.of(); + } + + @Override + public Set getPhysicalPlanOptimizers() { return ImmutableSet.of(new JdbcComputePushdown( functionManager, diff --git a/presto-docs/src/main/sphinx/connector.rst b/presto-docs/src/main/sphinx/connector.rst index 677437f35e26a..1479993ceafd9 100644 --- a/presto-docs/src/main/sphinx/connector.rst +++ b/presto-docs/src/main/sphinx/connector.rst @@ -22,6 +22,7 @@ from different data sources. connector/memory connector/mongodb connector/mysql + connector/pinot connector/postgresql connector/redis connector/redshift diff --git a/presto-docs/src/main/sphinx/connector/pinot.rst b/presto-docs/src/main/sphinx/connector/pinot.rst new file mode 100644 index 0000000000000..15c513c967755 --- /dev/null +++ b/presto-docs/src/main/sphinx/connector/pinot.rst @@ -0,0 +1,100 @@ +=============== +Pinot Connector +=============== + +The Pinot connector allows querying and creating tables in an external Pinot +database. This can be used to query pinot data or join pinot data with +something else. + +Configuration +------------- + +To configure the Pinot connector, create a catalog properties file +in ``etc/catalog`` named, for example, ``pinot.properties``, to +mount the Pinot connector as the ``pinot`` catalog. +Create the file with the following contents, replacing the +connection properties as appropriate for your setup: + +.. code-block:: none + + connector.name=pinot + pinot.controller-urls=controller_host1:9000,controller_host2:9000 + +Where the ``pinot.controller-urls`` property allows you to specify a +comma separated list of the pinot controller host/port pairs. + +Multiple Pinot Servers +^^^^^^^^^^^^^^^^^^^^^^ + +You can have as many catalogs as you need, so if you have additional +Pinot clusters, simply add another properties file to ``etc/catalog`` +with a different name (making sure it ends in ``.properties``). For +example, if you name the property file ``sales.properties``, Presto +will create a catalog named ``sales`` using the configured connector. + +Querying Pinot +-------------- + +The Pinot catalog exposes all pinot tables inside a flat schema. The +schema name is immaterial when querying but running ``SHOW SCHEMAS``, +will show just one schema entry of ``default``. + +The name of the pinot catalog is the catalog file you created above +without the ``.properties`` extension. + +For example, if you created a +file called ``mypinotcluster.properties``, you can see all the tables +in it using the command:: + + SHOW TABLES from mypinotcluster.default + +OR:: + + SHOW TABLES from mypinotcluster.foo + +Both of these commands will list all the tables in your pinot cluster. +This is because Pinot does not have a notion of schemas. + +Consider you have a table called ``clicks`` in the ``mypinotcluster``. +You can see a list of the columns in the ``clicks`` table using either +of the following:: + + DESCRIBE mypinotcluster.dontcare.clicks; + SHOW COLUMNS FROM mypinotcluster.dontcare.clicks; + +Finally, you can access the ``clicks`` table:: + + SELECT count(*) FROM mypinotcluster.default.clicks; + + +How the Pinot connector works +----------------------------- + +The connector tries to push the maximal subquery inferred from the +presto query into pinot. It can push down everything Pinot supports +including aggregations, group by, all UDFs etc. It generates the +correct Pinot PQL keeping Pinot's quirks in mind. + +By default, it sends aggregation and limit queries to the Pinot broker +and does a parallel scan for non-aggregation/non-limit queries. The +pinot broker queries create a single split that lets the Pinot broker +do the scatter gather. Whereas, in the parallel scan mode, there is +one split created for one-or-more Pinot segments and the Pinot servers +are directly contacted by the Presto servers (ie., the Pinot broker is +not involved in the parallel scan mode) + +There are a few configurations that control this behavior: + +* ``pinot.prefer-broker-queries``: This config is true by default. + Setting it to false will also create parallel plans for + aggregation and limit queries. +* ``pinot.forbid-segment-queries``: This config is false by default. + Setting it to true will forbid parallel querying and force all + querying to happen via the broker. +* ``pinot.non-aggregate-limit-for-broker-queries``: To prevent + overwhelming the broker, the connector only allows querying the + pinot broker for ``short`` queries. We define a ``short`` query to + be either an aggregation (or group-by) query or a query with a limit + less than the value configured for + ``pinot.non-aggregate-limit-for-broker-queries``. The default value + for this limit is 25K rows. diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/rule/HivePlanOptimizerProvider.java b/presto-hive/src/main/java/com/facebook/presto/hive/rule/HivePlanOptimizerProvider.java index f149c4eee8497..22ba73b70c439 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/rule/HivePlanOptimizerProvider.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/rule/HivePlanOptimizerProvider.java @@ -23,7 +23,13 @@ public class HivePlanOptimizerProvider implements ConnectorPlanOptimizerProvider { @Override - public Set getConnectorPlanOptimizers() + public Set getLogicalPlanOptimizers() + { + return ImmutableSet.of(); + } + + @Override + public Set getPhysicalPlanOptimizers() { return ImmutableSet.of(); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/ConnectorPlanOptimizerManager.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/ConnectorPlanOptimizerManager.java index 5754e8ad2f6df..6be197f8e92b2 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/ConnectorPlanOptimizerManager.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/ConnectorPlanOptimizerManager.java @@ -15,6 +15,7 @@ import com.facebook.presto.spi.ConnectorId; import com.facebook.presto.spi.ConnectorPlanOptimizer; +import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.connector.ConnectorPlanOptimizerProvider; import com.google.common.collect.ImmutableMap; @@ -24,6 +25,7 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import static com.facebook.presto.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.collect.Maps.transformValues; import static java.util.Objects.requireNonNull; @@ -43,8 +45,20 @@ public void addPlanOptimizerProvider(ConnectorId connectorId, ConnectorPlanOptim "ConnectorPlanOptimizerProvider for connector '%s' is already registered", connectorId); } - public Map> getOptimizers() + public Map> getOptimizers(PlanPhase phase) { - return ImmutableMap.copyOf(transformValues(planOptimizerProviders, ConnectorPlanOptimizerProvider::getConnectorPlanOptimizers)); + switch (phase) { + case LOGICAL: + return ImmutableMap.copyOf(transformValues(planOptimizerProviders, ConnectorPlanOptimizerProvider::getLogicalPlanOptimizers)); + case PHYSICAL: + return ImmutableMap.copyOf(transformValues(planOptimizerProviders, ConnectorPlanOptimizerProvider::getPhysicalPlanOptimizers)); + default: + throw new PrestoException(GENERIC_INTERNAL_ERROR, "Unknown plan phase " + phase); + } + } + + public enum PlanPhase + { + LOGICAL, PHYSICAL } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanOptimizers.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanOptimizers.java index 4c7e5f1a5b981..eabeb52348a01 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanOptimizers.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanOptimizers.java @@ -136,6 +136,9 @@ import java.util.List; import java.util.Set; +import static com.facebook.presto.sql.planner.ConnectorPlanOptimizerManager.PlanPhase.LOGICAL; +import static com.facebook.presto.sql.planner.ConnectorPlanOptimizerManager.PlanPhase.PHYSICAL; + public class PlanOptimizers { private final List optimizers; @@ -464,6 +467,10 @@ public PlanOptimizers( new TranslateExpressions(metadata, sqlParser).rules())); // After this point, all planNodes should not contain OriginalExpression + // TODO: move PushdownSubfields below this rule + // Pass a supplier so that we pickup connector optimizers that are installed later + builder.add(new ApplyConnectorOptimization(() -> planOptimizerManager.getOptimizers(LOGICAL))); + if (!forceSingleNode) { builder.add(new ReplicateSemiJoinInDelete()); // Must run before AddExchanges builder.add((new IterativeOptimizer( @@ -532,11 +539,8 @@ public PlanOptimizers( new AddIntermediateAggregations(), new RemoveRedundantIdentityProjections()))); - // TODO: Do not move other PlanNode to SPI until ApplyConnectorOptimization is moved to the end of logical planning (i.e., where AddExchanges lives) - // TODO: Run PruneUnreferencedOutputs and UnaliasSymbolReferences once we have cleaned it up - // Pass a supplier so that we pickup connector optimizers that are installed later builder.add( - new ApplyConnectorOptimization(planOptimizerManager::getOptimizers), + new ApplyConnectorOptimization(() -> planOptimizerManager.getOptimizers(PHYSICAL)), new IterativeOptimizer( ruleStats, statsCalculator, diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/PlanBuilder.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/PlanBuilder.java index 465f941e5ebe7..2816c13771a4f 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/PlanBuilder.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/PlanBuilder.java @@ -889,4 +889,9 @@ public TypeProvider getTypes() { return TypeProvider.viewOf(variables); } + + public PlanNodeIdAllocator getIdAllocator() + { + return idAllocator; + } } diff --git a/presto-pinot-toolkit/pom.xml b/presto-pinot-toolkit/pom.xml new file mode 100644 index 0000000000000..bda3f8bdba615 --- /dev/null +++ b/presto-pinot-toolkit/pom.xml @@ -0,0 +1,188 @@ + + + 4.0.0 + + com.facebook.presto + presto-root + 0.229-SNAPSHOT + + + presto-pinot-toolkit + presto-pinot-toolkit + Presto - Pinot Toolkit library + jar + + + ${project.parent.basedir} + + + + + com.facebook.presto.pinot + pinot-driver + + + + com.google.code.findbugs + jsr305 + + + + com.facebook.airlift + stats + + + + org.weakref + jmxutils + + + + com.facebook.airlift + bootstrap + + + + com.facebook.airlift + json + + + + com.facebook.airlift + log + + + + com.facebook.airlift + configuration + + + + io.airlift + units + provided + + + + com.google.guava + guava + + + + com.google.inject + guice + + + + javax.validation + validation-api + + + + javax.inject + javax.inject + + + + + com.facebook.presto + presto-spi + provided + + + + org.openjdk.jol + jol-core + provided + + + + io.airlift + slice + provided + + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + + com.fasterxml.jackson.core + jackson-databind + + + + joda-time + joda-time + + + + + com.facebook.presto + presto-main + test + + + + com.facebook.presto + presto-expressions + + + + com.facebook.presto + presto-main + test-jar + test + + + + org.testng + testng + test + + + + com.facebook.airlift + testing + test + + + + com.facebook.airlift + http-server + test + + + + com.facebook.airlift + node + test + + + + com.facebook.presto + presto-parser + test + + + + javax.servlet + javax.servlet-api + test + + + + com.facebook.airlift + http-client + + + + com.facebook.airlift + concurrent + + + diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/ForPinot.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/ForPinot.java new file mode 100644 index 0000000000000..e570b18583727 --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/ForPinot.java @@ -0,0 +1,31 @@ +/* + * 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.pinot; + +import javax.inject.Qualifier; + +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.FIELD; +import static java.lang.annotation.ElementType.METHOD; +import static java.lang.annotation.ElementType.PARAMETER; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +@Retention(RUNTIME) +@Target({FIELD, PARAMETER, METHOD}) +@Qualifier +public @interface ForPinot +{ +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotBrokerPageSource.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotBrokerPageSource.java new file mode 100644 index 0000000000000..62f94d5fdba09 --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotBrokerPageSource.java @@ -0,0 +1,420 @@ +/* + * 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.pinot; + +import com.facebook.airlift.http.client.Request; +import com.facebook.presto.pinot.query.PinotQueryGenerator.GeneratedPql; +import com.facebook.presto.spi.ConnectorPageSource; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.Page; +import com.facebook.presto.spi.PageBuilder; +import com.facebook.presto.spi.block.BlockBuilder; +import com.facebook.presto.spi.type.BigintType; +import com.facebook.presto.spi.type.BooleanType; +import com.facebook.presto.spi.type.DecimalType; +import com.facebook.presto.spi.type.DoubleType; +import com.facebook.presto.spi.type.FixedWidthType; +import com.facebook.presto.spi.type.IntegerType; +import com.facebook.presto.spi.type.SmallintType; +import com.facebook.presto.spi.type.TimestampType; +import com.facebook.presto.spi.type.TinyintType; +import com.facebook.presto.spi.type.Type; +import com.facebook.presto.spi.type.VarcharType; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; + +import java.io.IOException; +import java.net.URI; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +import static com.facebook.presto.pinot.PinotErrorCode.PINOT_DECODE_ERROR; +import static com.facebook.presto.pinot.PinotErrorCode.PINOT_EXCEPTION; +import static com.facebook.presto.pinot.PinotErrorCode.PINOT_INSUFFICIENT_SERVER_RESPONSE; +import static com.facebook.presto.pinot.PinotErrorCode.PINOT_UNEXPECTED_RESPONSE; +import static com.facebook.presto.pinot.PinotErrorCode.PINOT_UNSUPPORTED_COLUMN_TYPE; +import static com.facebook.presto.pinot.PinotUtils.doWithRetries; +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; +import static java.lang.Boolean.parseBoolean; +import static java.lang.Long.parseLong; +import static java.util.Objects.requireNonNull; + +public class PinotBrokerPageSource + implements ConnectorPageSource +{ + private static final String REQUEST_PAYLOAD_TEMPLATE = "{\"pql\" : \"%s\" }"; + private static final String QUERY_URL_TEMPLATE = "http://%s/query"; + + private static final String PINOT_INFINITY = "∞"; + private static final String PINOT_POSITIVE_INFINITY = "+" + PINOT_INFINITY; + private static final String PINOT_NEGATIVE_INFINITY = "-" + PINOT_INFINITY; + + private static final Double PRESTO_INFINITY = Double.POSITIVE_INFINITY; + private static final Double PRESTO_NEGATIVE_INFINITY = Double.NEGATIVE_INFINITY; + + private final GeneratedPql brokerPql; + private final PinotConfig pinotConfig; + private final List columnHandles; + private final PinotClusterInfoFetcher clusterInfoFetcher; + private final ConnectorSession session; + private final ObjectMapper objectMapper; + + private boolean finished; + private long readTimeNanos; + private long completedBytes; + + public PinotBrokerPageSource( + PinotConfig pinotConfig, + ConnectorSession session, + GeneratedPql brokerPql, + List columnHandles, + PinotClusterInfoFetcher clusterInfoFetcher, + ObjectMapper objectMapper) + { + this.pinotConfig = requireNonNull(pinotConfig, "pinot config is null"); + this.brokerPql = requireNonNull(brokerPql, "broker is null"); + this.clusterInfoFetcher = requireNonNull(clusterInfoFetcher, "cluster info fetcher is null"); + this.columnHandles = ImmutableList.copyOf(columnHandles); + this.session = requireNonNull(session, "session is null"); + this.objectMapper = requireNonNull(objectMapper, "object mapper is null"); + } + + private static Double parseDouble(String value) + { + try { + return Double.valueOf(value); + } + catch (NumberFormatException ne) { + switch (value) { + case PINOT_INFINITY: + case PINOT_POSITIVE_INFINITY: + return PRESTO_INFINITY; + case PINOT_NEGATIVE_INFINITY: + return PRESTO_NEGATIVE_INFINITY; + } + throw new PinotException(PINOT_DECODE_ERROR, Optional.empty(), "Cannot decode double value from pinot " + value, ne); + } + } + + private void setValue(Type type, BlockBuilder blockBuilder, String value) + { + if (value == null) { + blockBuilder.appendNull(); + return; + } + if (!(type instanceof FixedWidthType) && !(type instanceof VarcharType)) { + throw new PinotException(PINOT_UNSUPPORTED_COLUMN_TYPE, Optional.empty(), "type '" + type + "' not supported"); + } + if (type instanceof FixedWidthType) { + completedBytes += ((FixedWidthType) type).getFixedSize(); + if (type instanceof BigintType) { + type.writeLong(blockBuilder, parseDouble(value).longValue()); + } + else if (type instanceof IntegerType) { + blockBuilder.writeInt(parseDouble(value).intValue()); + } + else if (type instanceof TinyintType) { + blockBuilder.writeByte(parseDouble(value).byteValue()); + } + else if (type instanceof SmallintType) { + blockBuilder.writeShort(parseDouble(value).shortValue()); + } + else if (type instanceof BooleanType) { + type.writeBoolean(blockBuilder, parseBoolean(value)); + } + else if (type instanceof DecimalType || type instanceof DoubleType) { + type.writeDouble(blockBuilder, parseDouble(value)); + } + else if (type instanceof TimestampType) { + type.writeLong(blockBuilder, parseLong(value)); + } + else { + throw new PinotException(PINOT_UNSUPPORTED_COLUMN_TYPE, Optional.empty(), "type '" + type + "' not supported"); + } + } + else { + Slice slice = Slices.utf8Slice(value); + blockBuilder.writeBytes(slice, 0, slice.length()).closeEntry(); + completedBytes += slice.length(); + } + } + + private void setValuesForGroupby( + List blockBuilders, + List types, + int numGroupByClause, + JsonNode group, + String[] values) + { + for (int i = 0; i < group.size(); i++) { + setValue(types.get(i), blockBuilders.get(i), group.get(i).asText()); + } + for (int i = 0; i < values.length; i++) { + int metricColumnIndex = i + numGroupByClause; + if (metricColumnIndex < blockBuilders.size()) { + setValue(types.get(metricColumnIndex), blockBuilders.get(metricColumnIndex), values[i]); + } + } + } + + @Override + public long getCompletedBytes() + { + return completedBytes; + } + + @Override + public long getCompletedPositions() + { + return 0; // not available + } + + @Override + public long getReadTimeNanos() + { + return readTimeNanos; + } + + @Override + public boolean isFinished() + { + return finished; + } + + @Override + public Page getNextPage() + { + if (finished) { + return null; + } + + long start = System.nanoTime(); + try { + List expectedTypes = columnHandles.stream() + .map(PinotColumnHandle::getDataType) + .collect(Collectors.toList()); + PageBuilder pageBuilder = new PageBuilder(expectedTypes); + ImmutableList.Builder columnBlockBuilders = ImmutableList.builder(); + ImmutableList.Builder columnTypes = ImmutableList.builder(); + for (int i : brokerPql.getExpectedColumnIndices()) { + if (i == -1) { + continue; + } + BlockBuilder blockBuilder = pageBuilder.getBlockBuilder(i); + columnBlockBuilders.add(blockBuilder); + columnTypes.add(expectedTypes.get(i)); + } + + int counter = issuePqlAndPopulate( + brokerPql.getTable(), + brokerPql.getPql(), + brokerPql.getGroupByClauses(), + columnBlockBuilders.build(), + columnTypes.build()); + pageBuilder.declarePositions(counter); + Page page = pageBuilder.build(); + + // TODO: Implement chunking if the result set is ginormous + finished = true; + + return page; + } + finally { + readTimeNanos += System.nanoTime() - start; + } + } + + private int issuePqlAndPopulate( + String table, + String pql, + int numGroupByClause, + List blockBuilders, + List types) + { + return doWithRetries(PinotSessionProperties.getPinotRetryCount(session), (retryNumber) -> { + String queryHost; + Optional rpcService; + if (pinotConfig.getRestProxyUrl() != null) { + queryHost = pinotConfig.getRestProxyUrl(); + rpcService = Optional.ofNullable(pinotConfig.getRestProxyServiceForQuery()); + } + else { + queryHost = clusterInfoFetcher.getBrokerHost(table); + rpcService = Optional.empty(); + } + Request.Builder builder = Request.Builder + .preparePost() + .setUri(URI.create(String.format(QUERY_URL_TEMPLATE, queryHost))); + String body = clusterInfoFetcher.doHttpActionWithHeaders(builder, Optional.of(String.format(REQUEST_PAYLOAD_TEMPLATE, pql)), rpcService); + + return populateFromPqlResults(pql, numGroupByClause, blockBuilders, types, body); + }); + } + + @VisibleForTesting + public int populateFromPqlResults( + String pql, + int numGroupByClause, + List blockBuilders, + List types, + String body) + { + JsonNode jsonBody; + + try { + jsonBody = objectMapper.readTree(body); + } + catch (IOException e) { + throw new PinotException(PINOT_UNEXPECTED_RESPONSE, Optional.of(pql), "Couldn't parse response", e); + } + + JsonNode numServersResponded = jsonBody.get("numServersResponded"); + JsonNode numServersQueried = jsonBody.get("numServersQueried"); + + if (numServersQueried == null || numServersResponded == null || numServersQueried.asInt() > numServersResponded.asInt()) { + throw new PinotException( + PINOT_INSUFFICIENT_SERVER_RESPONSE, + Optional.of(pql), + String.format("Only %s out of %s servers responded for query %s", numServersResponded.asInt(), numServersQueried.asInt(), pql)); + } + + JsonNode exceptions = jsonBody.get("exceptions"); + if (exceptions != null && exceptions.isArray() && exceptions.size() > 0) { + // Pinot is known to return exceptions with benign errorcodes like 200 + // so we treat any exception as an error + throw new PinotException( + PINOT_EXCEPTION, + Optional.of(pql), + String.format("Query %s encountered exception %s", pql, exceptions.get(0))); + } + + JsonNode aggregationResults = jsonBody.get("aggregationResults"); + JsonNode selectionResults = jsonBody.get("selectionResults"); + + int rowCount; + if (aggregationResults != null && aggregationResults.isArray()) { + // This is map is populated only when we have multiple aggregates with a group by + checkState(aggregationResults.size() >= 1, "Expected at least one metric to be present"); + Map groupToValue = aggregationResults.size() == 1 || numGroupByClause == 0 ? null : new HashMap<>(); + rowCount = 0; + String[] singleAggregation = new String[1]; + Boolean seenGroupByResult = null; + for (int aggregationIndex = 0; aggregationIndex < aggregationResults.size(); aggregationIndex++) { + JsonNode result = aggregationResults.get(aggregationIndex); + + JsonNode metricValuesForEachGroup = result.get("groupByResult"); + + if (metricValuesForEachGroup != null) { + checkState(seenGroupByResult == null || seenGroupByResult); + seenGroupByResult = true; + checkState(numGroupByClause > 0, "Expected having non zero group by clauses"); + JsonNode groupByColumns = checkNotNull(result.get("groupByColumns"), "groupByColumns missing in %s", pql); + if (groupByColumns.size() != numGroupByClause) { + throw new PinotException( + PINOT_UNEXPECTED_RESPONSE, + Optional.of(pql), + String.format("Expected %d gby columns but got %s instead from pinot", numGroupByClause, groupByColumns)); + } + + // group by aggregation + for (int groupByIndex = 0; groupByIndex < metricValuesForEachGroup.size(); groupByIndex++) { + JsonNode row = metricValuesForEachGroup.get(groupByIndex); + JsonNode group = row.get("group"); + if (group == null || !group.isArray() || group.size() != numGroupByClause) { + throw new PinotException( + PINOT_UNEXPECTED_RESPONSE, + Optional.of(pql), + String.format("Expected %d group by columns but got only a group of size %d (%s)", numGroupByClause, group.size(), group)); + } + if (groupToValue == null) { + singleAggregation[0] = row.get("value").asText(); + setValuesForGroupby(blockBuilders, types, numGroupByClause, group, singleAggregation); + rowCount++; + } + else { + groupToValue.computeIfAbsent(group, (ignored) -> new String[aggregationResults.size()])[aggregationIndex] = row.get("value").asText(); + } + } + } + else { + checkState(seenGroupByResult == null || !seenGroupByResult); + seenGroupByResult = false; + // simple aggregation + // TODO: Validate that this is expected semantically + checkState(numGroupByClause == 0, "Expected no group by columns in pinot"); + setValue(types.get(aggregationIndex), blockBuilders.get(aggregationIndex), result.get("value").asText()); + rowCount = 1; + } + } + + if (groupToValue != null) { + checkState(rowCount == 0, "Row count shouldn't have changed from zero"); + groupToValue.forEach((group, values) -> setValuesForGroupby(blockBuilders, types, numGroupByClause, group, values)); + rowCount = groupToValue.size(); + } + } + else if (selectionResults != null) { + JsonNode columns = selectionResults.get("columns"); + JsonNode results = selectionResults.get("results"); + if (columns == null || results == null || !columns.isArray() || !results.isArray() || columns.size() != blockBuilders.size()) { + throw new PinotException( + PINOT_UNEXPECTED_RESPONSE, + Optional.of(pql), + String.format("Columns and results expected for %s, expected %d columns but got %d", pql, blockBuilders.size(), columns == null ? 0 : columns.size())); + } + for (int rowNumber = 0; rowNumber < results.size(); ++rowNumber) { + JsonNode result = results.get(rowNumber); + if (result == null || result.size() != blockBuilders.size()) { + throw new PinotException( + PINOT_UNEXPECTED_RESPONSE, + Optional.of(pql), + String.format("Expected row of %d columns", blockBuilders.size())); + } + for (int columnNumber = 0; columnNumber < blockBuilders.size(); columnNumber++) { + setValue(types.get(columnNumber), blockBuilders.get(columnNumber), result.get(columnNumber).asText()); + } + } + rowCount = results.size(); + } + else { + throw new PinotException( + PINOT_UNEXPECTED_RESPONSE, + Optional.of(pql), + "Expected one of aggregationResults or selectionResults to be present"); + } + + checkState(rowCount >= 0, "Expected row count to be initialized"); + return rowCount; + } + + @Override + public long getSystemMemoryUsage() + { + return 0; + } + + @Override + public void close() + { + finished = true; + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotClusterInfoFetcher.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotClusterInfoFetcher.java new file mode 100644 index 0000000000000..8189029dd5997 --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotClusterInfoFetcher.java @@ -0,0 +1,430 @@ +/* + * 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.pinot; + +import com.facebook.airlift.http.client.HttpClient; +import com.facebook.airlift.http.client.Request; +import com.facebook.airlift.http.client.StaticBodyGenerator; +import com.facebook.airlift.http.client.StringResponseHandler; +import com.facebook.airlift.json.JsonCodec; +import com.facebook.airlift.json.JsonCodecBinder; +import com.facebook.airlift.log.Logger; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Ticker; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.net.HttpHeaders; +import org.apache.pinot.common.data.Schema; + +import javax.inject.Inject; + +import java.net.URI; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Random; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import static com.facebook.airlift.http.client.StringResponseHandler.createStringResponseHandler; +import static com.facebook.presto.pinot.PinotErrorCode.PINOT_INVALID_CONFIGURATION; +import static com.facebook.presto.pinot.PinotErrorCode.PINOT_UNABLE_TO_FIND_BROKER; +import static com.facebook.presto.pinot.PinotErrorCode.PINOT_UNEXPECTED_RESPONSE; +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; +import static org.apache.pinot.common.config.TableNameBuilder.extractRawTableName; + +public class PinotClusterInfoFetcher +{ + private static final Logger log = Logger.get(PinotClusterInfoFetcher.class); + private static final String APPLICATION_JSON = "application/json"; + private static final Pattern BROKER_PATTERN = Pattern.compile("Broker_(.*)_(\\d+)"); + + private static final String GET_ALL_TABLES_API_TEMPLATE = "tables"; + private static final String TABLE_INSTANCES_API_TEMPLATE = "tables/%s/instances"; + private static final String TABLE_SCHEMA_API_TEMPLATE = "tables/%s/schema"; + private static final String ROUTING_TABLE_API_TEMPLATE = "debug/routingTable/%s"; + private static final String TIME_BOUNDARY_API_TEMPLATE = "debug/timeBoundary/%s"; + + private final PinotConfig pinotConfig; + private final PinotMetrics pinotMetrics; + private final HttpClient httpClient; + + private final Ticker ticker = Ticker.systemTicker(); + + private final LoadingCache> brokersForTableCache; + + private final JsonCodec tablesJsonCodec; + private final JsonCodec brokersForTableJsonCodec; + private final JsonCodec routingTablesJsonCodec; + private final JsonCodec timeBoundaryJsonCodec; + + @Inject + public PinotClusterInfoFetcher( + PinotConfig pinotConfig, + PinotMetrics pinotMetrics, + @ForPinot HttpClient httpClient, + JsonCodec tablesJsonCodec, + JsonCodec brokersForTableJsonCodec, + JsonCodec routingTablesJsonCodec, + JsonCodec timeBoundaryJsonCodec) + { + this.brokersForTableJsonCodec = requireNonNull(brokersForTableJsonCodec, "brokers for table json codec is null"); + this.routingTablesJsonCodec = requireNonNull(routingTablesJsonCodec, "routing tables json codec is null"); + this.timeBoundaryJsonCodec = requireNonNull(timeBoundaryJsonCodec, "time boundary json codec is null"); + final long cacheExpiryMs = pinotConfig.getMetadataCacheExpiry().roundTo(TimeUnit.MILLISECONDS); + this.tablesJsonCodec = requireNonNull(tablesJsonCodec, "json codec is null"); + + this.pinotConfig = requireNonNull(pinotConfig, "pinotConfig is null"); + this.pinotMetrics = requireNonNull(pinotMetrics, "pinotMetrics is null"); + this.httpClient = requireNonNull(httpClient, "httpClient is null"); + this.brokersForTableCache = CacheBuilder.newBuilder() + .expireAfterWrite(cacheExpiryMs, TimeUnit.MILLISECONDS) + .build((CacheLoader.from(this::getAllBrokersForTable))); + } + + public static JsonCodecBinder addJsonBinders(JsonCodecBinder jsonCodecBinder) + { + jsonCodecBinder.bindJsonCodec(GetTables.class); + jsonCodecBinder.bindJsonCodec(BrokersForTable.InstancesInBroker.class); + jsonCodecBinder.bindJsonCodec(BrokersForTable.class); + jsonCodecBinder.bindJsonCodec(RoutingTables.class); + jsonCodecBinder.bindJsonCodec(RoutingTables.RoutingTableSnapshot.class); + jsonCodecBinder.bindJsonCodec(TimeBoundary.class); + return jsonCodecBinder; + } + + public String doHttpActionWithHeaders( + Request.Builder requestBuilder, + Optional requestBody, + Optional rpcService) + { + requestBuilder = requestBuilder + .setHeader(HttpHeaders.CONTENT_TYPE, APPLICATION_JSON) + .setHeader(HttpHeaders.ACCEPT, APPLICATION_JSON); + if (rpcService.isPresent()) { + requestBuilder + .setHeader(pinotConfig.getCallerHeaderParam(), pinotConfig.getCallerHeaderValue()) + .setHeader(pinotConfig.getServiceHeaderParam(), rpcService.get()); + } + if (requestBody.isPresent()) { + requestBuilder.setBodyGenerator(StaticBodyGenerator.createStaticBodyGenerator(requestBody.get(), StandardCharsets.UTF_8)); + } + pinotConfig.getExtraHttpHeaders().forEach(requestBuilder::setHeader); + Request request = requestBuilder.build(); + + long startTime = ticker.read(); + long duration; + StringResponseHandler.StringResponse response; + try { + response = httpClient.execute(request, createStringResponseHandler()); + } + finally { + duration = ticker.read() - startTime; + } + pinotMetrics.monitorRequest(request, response, duration, TimeUnit.NANOSECONDS); + String responseBody = response.getBody(); + if (PinotUtils.isValidPinotHttpResponseCode(response.getStatusCode())) { + return responseBody; + } + else { + throw new PinotException( + PinotErrorCode.PINOT_HTTP_ERROR, + Optional.empty(), + String.format( + "Unexpected response status: %d for request %s to url %s, with headers %s, full response %s", + response.getStatusCode(), + requestBody.orElse(""), + request.getUri(), + request.getHeaders(), + responseBody)); + } + } + + private String sendHttpGetToController(String path) + { + return doHttpActionWithHeaders( + Request.builder().prepareGet().setUri(URI.create(String.format("http://%s/%s", getControllerUrl(), path))), + Optional.empty(), + Optional.ofNullable(pinotConfig.getControllerRestService())); + } + + private String sendHttpGetToBroker(String table, String path) + { + return doHttpActionWithHeaders( + Request.builder().prepareGet().setUri(URI.create(String.format("http://%s/%s", getBrokerHost(table), path))), + Optional.empty(), + Optional.empty()); + } + + private String getControllerUrl() + { + List controllerUrls = pinotConfig.getControllerUrls(); + if (controllerUrls.isEmpty()) { + throw new PinotException(PINOT_INVALID_CONFIGURATION, Optional.empty(), "No pinot controllers specified"); + } + return controllerUrls.get(ThreadLocalRandom.current().nextInt(controllerUrls.size())); + } + + public static class GetTables + { + private final List tables; + + @JsonCreator + public GetTables(@JsonProperty("tables") List tables) + { + this.tables = tables; + } + + public List getTables() + { + return tables; + } + } + + public List getAllTables() + { + return tablesJsonCodec.fromJson(sendHttpGetToController(GET_ALL_TABLES_API_TEMPLATE)).getTables(); + } + + public Schema getTableSchema(String table) + throws Exception + { + String responseBody = sendHttpGetToController(String.format(TABLE_SCHEMA_API_TEMPLATE, table)); + return Schema.fromString(responseBody); + } + + public static class BrokersForTable + { + public static class InstancesInBroker + { + private final List instances; + + @JsonCreator + public InstancesInBroker(@JsonProperty("instances") List instances) + { + this.instances = instances; + } + + @JsonProperty("instances") + public List getInstances() + { + return instances; + } + } + + private final List brokers; + + @JsonCreator + public BrokersForTable(@JsonProperty("brokers") List brokers) + { + this.brokers = brokers; + } + + @JsonProperty("brokers") + public List getBrokers() + { + return brokers; + } + } + + @VisibleForTesting + List getAllBrokersForTable(String table) + { + String responseBody = sendHttpGetToController(String.format(TABLE_INSTANCES_API_TEMPLATE, table)); + ArrayList brokers = brokersForTableJsonCodec + .fromJson(responseBody) + .getBrokers() + .stream() + .flatMap(broker -> broker.getInstances().stream()) + .distinct() + .map(brokerToParse -> { + Matcher matcher = BROKER_PATTERN.matcher(brokerToParse); + if (matcher.matches() && matcher.groupCount() == 2) { + return matcher.group(1) + ":" + matcher.group(2); + } + else { + throw new PinotException( + PINOT_UNABLE_TO_FIND_BROKER, + Optional.empty(), + String.format("Cannot parse %s in the broker instance", brokerToParse)); + } + }) + .collect(Collectors.toCollection(() -> new ArrayList<>())); + Collections.shuffle(brokers); + return ImmutableList.copyOf(brokers); + } + + public String getBrokerHost(String table) + { + try { + List brokers = brokersForTableCache.get(table); + if (brokers.isEmpty()) { + throw new PinotException(PINOT_UNABLE_TO_FIND_BROKER, Optional.empty(), "No valid brokers found for " + table); + } + return brokers.get(ThreadLocalRandom.current().nextInt(brokers.size())); + } + catch (ExecutionException e) { + Throwable throwable = e.getCause(); + if (throwable instanceof PinotException) { + throw (PinotException) throwable; + } + else { + throw new PinotException(PINOT_UNABLE_TO_FIND_BROKER, Optional.empty(), "Error when getting brokers for table " + table, throwable); + } + } + } + + public static class RoutingTables + { + public static class RoutingTableSnapshot + { + private final String tableName; + private final List>> routingTableEntries; + + @JsonCreator + public RoutingTableSnapshot( + @JsonProperty("tableName") String tableName, + @JsonProperty("routingTableEntries") List>> routingTableEntries) + { + this.tableName = requireNonNull(tableName, "table name is null"); + this.routingTableEntries = requireNonNull(routingTableEntries, "routing table entries is null"); + } + + @JsonProperty("tableName") + public String getTableName() + { + return tableName; + } + + @JsonProperty("routingTableEntries") + public List>> getRoutingTableEntries() + { + return routingTableEntries; + } + } + + private final List routingTableSnapshot; + + @JsonCreator + public RoutingTables(@JsonProperty("routingTableSnapshot") List routingTableSnapshot) + { + this.routingTableSnapshot = routingTableSnapshot; + } + + public List getRoutingTableSnapshot() + { + return routingTableSnapshot; + } + } + + public Map>> getRoutingTableForTable(String tableName) + { + ImmutableMap.Builder>> routingTableMap = ImmutableMap.builder(); + log.debug("Trying to get routingTable for %s from broker", tableName); + String responseBody = sendHttpGetToBroker(tableName, String.format(ROUTING_TABLE_API_TEMPLATE, tableName)); + routingTablesJsonCodec.fromJson(responseBody).getRoutingTableSnapshot().forEach(snapshot -> { + String tableNameWithType = snapshot.getTableName(); + // Response could contain info for tableName that matches the original table by prefix. + // e.g. when table name is "table1", response could contain routingTable for "table1_staging" + if (!tableName.equals(extractRawTableName(tableNameWithType))) { + log.debug("Ignoring routingTable for %s", tableNameWithType); + } + else { + List>> routingTableEntriesList = snapshot.getRoutingTableEntries(); + if (routingTableEntriesList.isEmpty()) { + throw new PinotException( + PINOT_UNEXPECTED_RESPONSE, + Optional.empty(), + String.format("Empty routingTableEntries for %s. RoutingTable: %s", tableName, responseBody)); + } + + // We are given multiple routing tables for a table, each with different segment to host assignments + // We pick one randomly, so that a retry may hit a different server + Map> routingTableEntries = routingTableEntriesList.get(new Random().nextInt(routingTableEntriesList.size())); + ImmutableMap.Builder> routingTableBuilder = ImmutableMap.builder(); + routingTableEntries.forEach((host, segments) -> { + List segmentsCopied = new ArrayList<>(segments); + Collections.shuffle(segmentsCopied); + routingTableBuilder.put(host, ImmutableList.copyOf(segmentsCopied)); + }); + routingTableMap.put(tableNameWithType, routingTableBuilder.build()); + } + }); + return routingTableMap.build(); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("pinotConfig", pinotConfig) + .toString(); + } + + public static class TimeBoundary + { + private final Optional onlineTimePredicate; + private final Optional offlineTimePredicate; + + public TimeBoundary() + { + this(null, null); + } + + @JsonCreator + public TimeBoundary( + @JsonProperty String timeColumnName, + @JsonProperty String timeColumnValue) + { + if (timeColumnName != null && timeColumnValue != null) { + offlineTimePredicate = Optional.of(format("%s < %s", timeColumnName, timeColumnValue)); + onlineTimePredicate = Optional.of(format("%s >= %s", timeColumnName, timeColumnValue)); + } + else { + onlineTimePredicate = Optional.empty(); + offlineTimePredicate = Optional.empty(); + } + } + + public Optional getOnlineTimePredicate() + { + return onlineTimePredicate; + } + + public Optional getOfflineTimePredicate() + { + return offlineTimePredicate; + } + } + + public TimeBoundary getTimeBoundaryForTable(String table) + { + String responseBody = sendHttpGetToBroker(table, String.format(TIME_BOUNDARY_API_TEMPLATE, table)); + return timeBoundaryJsonCodec.fromJson(responseBody); + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotColumn.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotColumn.java new file mode 100644 index 0000000000000..8c4f0a1af45f6 --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotColumn.java @@ -0,0 +1,78 @@ +/* + * 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.pinot; + +import com.facebook.presto.spi.type.Type; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Strings.isNullOrEmpty; +import static java.util.Objects.requireNonNull; + +public final class PinotColumn +{ + private final String name; + private final Type type; + + @JsonCreator + public PinotColumn( + @JsonProperty("name") String name, + @JsonProperty("type") Type type) + { + checkArgument(!isNullOrEmpty(name), "name is null or is empty"); + this.name = name; + this.type = requireNonNull(type, "type is null"); + } + + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty + public Type getType() + { + return type; + } + + @Override + public int hashCode() + { + return Objects.hash(name, type); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + PinotColumn other = (PinotColumn) obj; + return Objects.equals(this.name, other.name) && Objects.equals(this.type, other.type); + } + + @Override + public String toString() + { + return name + ":" + type; + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotColumnHandle.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotColumnHandle.java new file mode 100644 index 0000000000000..c655a704ff848 --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotColumnHandle.java @@ -0,0 +1,112 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.pinot; + +import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.ColumnMetadata; +import com.facebook.presto.spi.relation.VariableReferenceExpression; +import com.facebook.presto.spi.type.Type; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +public final class PinotColumnHandle + implements ColumnHandle +{ + private final String columnName; + private final Type dataType; + private final PinotColumnType type; + + public PinotColumnHandle( + VariableReferenceExpression variable, + PinotColumnType type) + { + this(variable.getName(), variable.getType(), type); + } + + @JsonCreator + public PinotColumnHandle( + @JsonProperty("columnName") String columnName, + @JsonProperty("dataType") Type dataType, + @JsonProperty("type") PinotColumnType type) + { + this.columnName = requireNonNull(columnName, "column name is null"); + this.dataType = requireNonNull(dataType, "data type name is null"); + this.type = requireNonNull(type, "type is null"); + } + + @JsonProperty("columnName") + public String getColumnName() + { + return columnName; + } + + @JsonProperty("dataType") + public Type getDataType() + { + return dataType; + } + + @JsonProperty + public PinotColumnType getType() + { + return type; + } + + public ColumnMetadata getColumnMetadata() + { + return new ColumnMetadata(getColumnName(), getDataType()); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + PinotColumnHandle that = (PinotColumnHandle) o; + return Objects.equals(getColumnName(), that.getColumnName()); + } + + @Override + public int hashCode() + { + return Objects.hash(columnName); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("columnName", columnName) + .add("dataType", dataType) + .add("type", type) + .toString(); + } + + public enum PinotColumnType + { + REGULAR, // refers to the column in table + DERIVED, // refers to a derived column that is created after a pushdown expression + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotColumnMetadata.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotColumnMetadata.java new file mode 100644 index 0000000000000..b19d08094c59e --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotColumnMetadata.java @@ -0,0 +1,68 @@ +/* + * 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.pinot; + +import com.facebook.presto.spi.ColumnMetadata; +import com.facebook.presto.spi.type.Type; + +import java.util.Objects; + +import static java.util.Locale.ENGLISH; +import static java.util.Objects.requireNonNull; + +public class PinotColumnMetadata + extends ColumnMetadata +{ + // We need to preserve the case sensitivity of the column, store it here as the super class stores the value after lower-casing it + private final String name; + + public PinotColumnMetadata(String name, Type type) + { + super(requireNonNull(name, "name is null"), requireNonNull(type, "type is null")); + this.name = name; + } + + @Override + public String getName() + { + return name.toLowerCase(ENGLISH); + } + + public String getPinotName() + { + return name; + } + + @Override + public int hashCode() + { + return Objects.hash(name, getType(), getComment(), isHidden()); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + PinotColumnMetadata other = (PinotColumnMetadata) obj; + return Objects.equals(this.name, other.name) && + Objects.equals(this.getType(), other.getType()) && + Objects.equals(this.getComment(), other.getComment()) && + Objects.equals(this.isHidden(), other.isHidden()); + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotColumnUtils.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotColumnUtils.java new file mode 100644 index 0000000000000..a1f896564bdd3 --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotColumnUtils.java @@ -0,0 +1,73 @@ +/* + * 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.pinot; + +import com.facebook.presto.spi.type.BigintType; +import com.facebook.presto.spi.type.BooleanType; +import com.facebook.presto.spi.type.DoubleType; +import com.facebook.presto.spi.type.IntegerType; +import com.facebook.presto.spi.type.Type; +import com.facebook.presto.spi.type.VarcharType; +import org.apache.pinot.common.data.FieldSpec; +import org.apache.pinot.common.data.FieldSpec.DataType; +import org.apache.pinot.common.data.Schema; + +import java.util.List; +import java.util.Optional; + +import static com.facebook.presto.pinot.PinotErrorCode.PINOT_UNSUPPORTED_COLUMN_TYPE; +import static com.google.common.collect.ImmutableList.toImmutableList; + +public class PinotColumnUtils +{ + private PinotColumnUtils() + { + } + + public static List getPinotColumnsForPinotSchema(Schema pinotTableSchema) + { + return pinotTableSchema.getColumnNames().stream() + .filter(columnName -> !columnName.startsWith("$")) // Hidden columns starts with "$", ignore them as we can't use them in PQL + .map(columnName -> new PinotColumn(columnName, getPrestoTypeFromPinotType(pinotTableSchema.getFieldSpecFor(columnName)))) + .collect(toImmutableList()); + } + + public static Type getPrestoTypeFromPinotType(FieldSpec field) + { + if (field.isSingleValueField()) { + return getPrestoTypeFromPinotType(field.getDataType()); + } + return VarcharType.VARCHAR; + } + + public static Type getPrestoTypeFromPinotType(DataType dataType) + { + switch (dataType) { + case BOOLEAN: + return BooleanType.BOOLEAN; + case DOUBLE: + case FLOAT: + return DoubleType.DOUBLE; + case INT: + return IntegerType.INTEGER; + case LONG: + return BigintType.BIGINT; + case STRING: + return VarcharType.VARCHAR; + default: + break; + } + throw new PinotException(PINOT_UNSUPPORTED_COLUMN_TYPE, Optional.empty(), "Not support type conversion for pinot data type: " + dataType); + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotConfig.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotConfig.java new file mode 100644 index 0000000000000..f4b2aa793a05c --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotConfig.java @@ -0,0 +1,419 @@ +/* + * 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.pinot; + +import com.facebook.airlift.configuration.Config; +import com.google.common.base.Splitter; +import com.google.common.base.Splitter.MapSplitter; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.airlift.units.Duration; +import io.airlift.units.MinDuration; + +import javax.annotation.Nullable; +import javax.validation.constraints.NotNull; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import static com.google.common.base.Preconditions.checkArgument; + +public class PinotConfig +{ + public static final int DEFAULT_LIMIT_LARGE_FOR_SEGMENT = Integer.MAX_VALUE; + public static final int DEFAULT_MAX_BACKLOG_PER_SERVER = 30; + public static final int DEFAULT_MAX_CONNECTIONS_PER_SERVER = 30; + public static final int DEFAULT_MIN_CONNECTIONS_PER_SERVER = 10; + public static final int DEFAULT_THREAD_POOL_SIZE = 30; + public static final int DEFAULT_NON_AGGREGATE_LIMIT_FOR_BROKER_QUERIES = 25_000; + + // There is a perf penalty of having a large topN since the structures are allocated to this size + // So size this judiciously + public static final int DEFAULT_TOPN_LARGE = 10_000; + + private static final Duration DEFAULT_IDLE_TIMEOUT = new Duration(5, TimeUnit.MINUTES); + private static final Duration DEFAULT_CONNECTION_TIMEOUT = new Duration(1, TimeUnit.MINUTES); + private static final int DEFAULT_ESTIMATED_SIZE_IN_BYTES_FOR_NON_NUMERIC_COLUMN = 20; + + private static final Splitter LIST_SPLITTER = Splitter.on(",").trimResults().omitEmptyStrings(); + private static final MapSplitter MAP_SPLITTER = Splitter.on(",").trimResults().omitEmptyStrings().withKeyValueSeparator(":"); + + private int maxConnectionsPerServer = DEFAULT_MAX_CONNECTIONS_PER_SERVER; + private String controllerRestService; + private String serviceHeaderParam = "RPC-Service"; + private String callerHeaderValue = "presto"; + private String callerHeaderParam = "RPC-Caller"; + + private List controllerUrls = ImmutableList.of(); + private String restProxyUrl; + private String restProxyServiceForQuery; + + private int limitLargeForSegment = DEFAULT_LIMIT_LARGE_FOR_SEGMENT; + private int topNLarge = DEFAULT_TOPN_LARGE; + + private Duration idleTimeout = DEFAULT_IDLE_TIMEOUT; + private Duration connectionTimeout = DEFAULT_CONNECTION_TIMEOUT; + + private int threadPoolSize = DEFAULT_THREAD_POOL_SIZE; + private int minConnectionsPerServer = DEFAULT_MIN_CONNECTIONS_PER_SERVER; + private int maxBacklogPerServer = DEFAULT_MAX_BACKLOG_PER_SERVER; + private int estimatedSizeInBytesForNonNumericColumn = DEFAULT_ESTIMATED_SIZE_IN_BYTES_FOR_NON_NUMERIC_COLUMN; + private Map extraHttpHeaders = ImmutableMap.of(); + private Duration metadataCacheExpiry = new Duration(2, TimeUnit.MINUTES); + + private boolean allowMultipleAggregations; + private boolean preferBrokerQueries = true; + private boolean forbidSegmentQueries; + private int numSegmentsPerSplit = 1; + private boolean ignoreEmptyResponses; + private int fetchRetryCount = 2; + private boolean useDateTrunc; + private int nonAggregateLimitForBrokerQueries = DEFAULT_NON_AGGREGATE_LIMIT_FOR_BROKER_QUERIES; + + @NotNull + public Map getExtraHttpHeaders() + { + return extraHttpHeaders; + } + + @Config("pinot.extra-http-headers") + public PinotConfig setExtraHttpHeaders(String headers) + { + extraHttpHeaders = ImmutableMap.copyOf(MAP_SPLITTER.split(headers)); + return this; + } + + @NotNull + public List getControllerUrls() + { + return controllerUrls; + } + + @Config("pinot.controller-urls") + public PinotConfig setControllerUrls(String controllerUrl) + { + this.controllerUrls = LIST_SPLITTER.splitToList(controllerUrl); + return this; + } + + @Nullable + public String getRestProxyUrl() + { + return restProxyUrl; + } + + @Config("pinot.rest-proxy-url") + public PinotConfig setRestProxyUrl(String restProxyUrl) + { + this.restProxyUrl = restProxyUrl; + return this; + } + + @NotNull + public String getControllerRestService() + { + return controllerRestService; + } + + @Config("pinot.controller-rest-service") + public PinotConfig setControllerRestService(String controllerRestService) + { + this.controllerRestService = controllerRestService; + return this; + } + + @NotNull + public boolean isAllowMultipleAggregations() + { + return allowMultipleAggregations; + } + + @Config("pinot.allow-multiple-aggregations") + public PinotConfig setAllowMultipleAggregations(boolean allowMultipleAggregations) + { + this.allowMultipleAggregations = allowMultipleAggregations; + return this; + } + + @NotNull + public int getLimitLargeForSegment() + { + return limitLargeForSegment; + } + + @Config("pinot.limit-large-for-segment") + public PinotConfig setLimitLargeForSegment(int limitLargeForSegment) + { + this.limitLargeForSegment = limitLargeForSegment; + return this; + } + + @NotNull + public int getTopNLarge() + { + return topNLarge; + } + + @Config("pinot.topn-large") + public PinotConfig setTopNLarge(int topNLarge) + { + this.topNLarge = topNLarge; + return this; + } + + @NotNull + public int getThreadPoolSize() + { + return threadPoolSize; + } + + @Config("pinot.thread-pool-size") + public PinotConfig setThreadPoolSize(int threadPoolSize) + { + this.threadPoolSize = threadPoolSize; + return this; + } + + @NotNull + public int getMinConnectionsPerServer() + { + return minConnectionsPerServer; + } + + @Config("pinot.min-connections-per-server") + public PinotConfig setMinConnectionsPerServer(int minConnectionsPerServer) + { + this.minConnectionsPerServer = minConnectionsPerServer; + return this; + } + + @NotNull + public int getMaxConnectionsPerServer() + { + return maxConnectionsPerServer; + } + + @Config("pinot.max-connections-per-server") + public PinotConfig setMaxConnectionsPerServer(int maxConnectionsPerServer) + { + this.maxConnectionsPerServer = maxConnectionsPerServer; + return this; + } + + @NotNull + public int getMaxBacklogPerServer() + { + return maxBacklogPerServer; + } + + @Config("pinot.max-backlog-per-server") + public PinotConfig setMaxBacklogPerServer(int maxBacklogPerServer) + { + this.maxBacklogPerServer = maxBacklogPerServer; + return this; + } + + @MinDuration("15s") + @NotNull + public Duration getIdleTimeout() + { + return idleTimeout; + } + + @Config("pinot.idle-timeout") + public PinotConfig setIdleTimeout(Duration idleTimeout) + { + this.idleTimeout = idleTimeout; + return this; + } + + @MinDuration("15s") + @NotNull + public Duration getConnectionTimeout() + { + return connectionTimeout; + } + + @Config("pinot.connection-timeout") + public PinotConfig setConnectionTimeout(Duration connectionTimeout) + { + this.connectionTimeout = connectionTimeout; + return this; + } + + @MinDuration("0s") + @NotNull + public Duration getMetadataCacheExpiry() + { + return metadataCacheExpiry; + } + + @Config("pinot.metadata-expiry") + public PinotConfig setMetadataCacheExpiry(Duration metadataCacheExpiry) + { + this.metadataCacheExpiry = metadataCacheExpiry; + return this; + } + + @NotNull + public int getEstimatedSizeInBytesForNonNumericColumn() + { + return estimatedSizeInBytesForNonNumericColumn; + } + + @Config("pinot.estimated-size-in-bytes-for-non-numeric-column") + public PinotConfig setEstimatedSizeInBytesForNonNumericColumn(int estimatedSizeInBytesForNonNumericColumn) + { + this.estimatedSizeInBytesForNonNumericColumn = estimatedSizeInBytesForNonNumericColumn; + return this; + } + + @NotNull + public String getServiceHeaderParam() + { + return serviceHeaderParam; + } + + @Config("pinot.service-header-param") + public PinotConfig setServiceHeaderParam(String serviceHeaderParam) + { + this.serviceHeaderParam = serviceHeaderParam; + return this; + } + + @NotNull + public String getCallerHeaderValue() + { + return callerHeaderValue; + } + + @Config("pinot.caller-header-value") + public PinotConfig setCallerHeaderValue(String callerHeaderValue) + { + this.callerHeaderValue = callerHeaderValue; + return this; + } + + @NotNull + public String getCallerHeaderParam() + { + return callerHeaderParam; + } + + @Config("pinot.caller-header-param") + public PinotConfig setCallerHeaderParam(String callerHeaderParam) + { + this.callerHeaderParam = callerHeaderParam; + return this; + } + + public boolean isPreferBrokerQueries() + { + return preferBrokerQueries; + } + + @Config("pinot.prefer-broker-queries") + public PinotConfig setPreferBrokerQueries(boolean preferBrokerQueries) + { + this.preferBrokerQueries = preferBrokerQueries; + return this; + } + + public boolean isForbidSegmentQueries() + { + return forbidSegmentQueries; + } + + @Config("pinot.forbid-segment-queries") + public PinotConfig setForbidSegmentQueries(boolean forbidSegmentQueries) + { + this.forbidSegmentQueries = forbidSegmentQueries; + return this; + } + + @Nullable + public String getRestProxyServiceForQuery() + { + return restProxyServiceForQuery; + } + + @Config("pinot.rest-proxy-service-for-query") + public PinotConfig setRestProxyServiceForQuery(String restProxyServiceForQuery) + { + this.restProxyServiceForQuery = restProxyServiceForQuery; + return this; + } + + public boolean isUseDateTrunc() + { + return useDateTrunc; + } + + @Config("pinot.use-date-trunc") + public PinotConfig setUseDateTrunc(boolean useDateTrunc) + { + this.useDateTrunc = useDateTrunc; + return this; + } + + public int getNumSegmentsPerSplit() + { + return this.numSegmentsPerSplit; + } + + @Config("pinot.num-segments-per-split") + public PinotConfig setNumSegmentsPerSplit(int numSegmentsPerSplit) + { + checkArgument(numSegmentsPerSplit > 0, "Number of segments per split must be more than zero"); + this.numSegmentsPerSplit = numSegmentsPerSplit; + return this; + } + + public boolean isIgnoreEmptyResponses() + { + return ignoreEmptyResponses; + } + + @Config("pinot.ignore-empty-responses") + public PinotConfig setIgnoreEmptyResponses(boolean ignoreEmptyResponses) + { + this.ignoreEmptyResponses = ignoreEmptyResponses; + return this; + } + + public int getFetchRetryCount() + { + return fetchRetryCount; + } + + @Config("pinot.fetch-retry-count") + public PinotConfig setFetchRetryCount(int fetchRetryCount) + { + this.fetchRetryCount = fetchRetryCount; + return this; + } + + public int getNonAggregateLimitForBrokerQueries() + { + return nonAggregateLimitForBrokerQueries; + } + + @Config("pinot.non-aggregate-limit-for-broker-queries") + public PinotConfig setNonAggregateLimitForBrokerQueries(int nonAggregateLimitForBrokerQueries) + { + this.nonAggregateLimitForBrokerQueries = nonAggregateLimitForBrokerQueries; + return this; + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotConnection.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotConnection.java new file mode 100644 index 0000000000000..970f6171d9ead --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotConnection.java @@ -0,0 +1,112 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.pinot; + +import com.facebook.presto.pinot.PinotClusterInfoFetcher.TimeBoundary; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; +import org.apache.pinot.common.data.Schema; + +import javax.inject.Inject; + +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; + +import static com.google.common.cache.CacheLoader.asyncReloading; +import static java.util.Objects.requireNonNull; + +public class PinotConnection +{ + private static final Object ALL_TABLES_CACHE_KEY = new Object(); + + private final LoadingCache> pinotTableColumnCache; + private final LoadingCache> allTablesCache; + private final PinotConfig pinotConfig; + private final PinotClusterInfoFetcher pinotClusterInfoFetcher; + + @Inject + public PinotConnection( + PinotClusterInfoFetcher pinotClusterInfoFetcher, + PinotConfig pinotConfig, + @ForPinot Executor executor) + { + this.pinotConfig = requireNonNull(pinotConfig, "pinot config"); + final long metadataCacheExpiryMillis = this.pinotConfig.getMetadataCacheExpiry().roundTo(TimeUnit.MILLISECONDS); + this.pinotClusterInfoFetcher = requireNonNull(pinotClusterInfoFetcher, "cluster info fetcher is null"); + this.allTablesCache = CacheBuilder.newBuilder() + .refreshAfterWrite(metadataCacheExpiryMillis, TimeUnit.MILLISECONDS) + .build(asyncReloading(CacheLoader.from(pinotClusterInfoFetcher::getAllTables), executor)); + + this.pinotTableColumnCache = + CacheBuilder.newBuilder() + .refreshAfterWrite(metadataCacheExpiryMillis, TimeUnit.MILLISECONDS) + .build(asyncReloading(new CacheLoader>() + { + @Override + public List load(String tableName) + throws Exception + { + Schema tablePinotSchema = pinotClusterInfoFetcher.getTableSchema(tableName); + return PinotColumnUtils.getPinotColumnsForPinotSchema(tablePinotSchema); + } + }, executor)); + + executor.execute(() -> this.allTablesCache.refresh(ALL_TABLES_CACHE_KEY)); + } + + private static V getFromCache(LoadingCache cache, K key) + { + V value = cache.getIfPresent(key); + if (value != null) { + return value; + } + try { + return cache.get(key); + } + catch (ExecutionException e) { + throw new PinotException(PinotErrorCode.PINOT_UNCLASSIFIED_ERROR, Optional.empty(), "Cannot fetch from cache " + key, e.getCause()); + } + } + + public List getTableNames() + { + return getFromCache(allTablesCache, ALL_TABLES_CACHE_KEY); + } + + public PinotTable getTable(String tableName) + { + List columns = getPinotColumnsForTable(tableName); + return new PinotTable(tableName, columns); + } + + private List getPinotColumnsForTable(String tableName) + { + return getFromCache(pinotTableColumnCache, tableName); + } + + public Map>> getRoutingTable(String tableName) + { + return pinotClusterInfoFetcher.getRoutingTableForTable(tableName); + } + + public TimeBoundary getTimeBoundary(String tableName) + { + return pinotClusterInfoFetcher.getTimeBoundaryForTable(tableName); + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotConnector.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotConnector.java new file mode 100644 index 0000000000000..73724f6c0ad0f --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotConnector.java @@ -0,0 +1,134 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.pinot; + +import com.facebook.airlift.bootstrap.LifeCycleManager; +import com.facebook.airlift.log.Logger; +import com.facebook.presto.spi.ConnectorPlanOptimizer; +import com.facebook.presto.spi.connector.Connector; +import com.facebook.presto.spi.connector.ConnectorMetadata; +import com.facebook.presto.spi.connector.ConnectorNodePartitioningProvider; +import com.facebook.presto.spi.connector.ConnectorPageSourceProvider; +import com.facebook.presto.spi.connector.ConnectorPlanOptimizerProvider; +import com.facebook.presto.spi.connector.ConnectorSplitManager; +import com.facebook.presto.spi.connector.ConnectorTransactionHandle; +import com.facebook.presto.spi.session.PropertyMetadata; +import com.facebook.presto.spi.transaction.IsolationLevel; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; + +import javax.inject.Inject; + +import java.util.List; +import java.util.Set; + +import static com.facebook.presto.pinot.PinotTransactionHandle.INSTANCE; +import static java.util.Objects.requireNonNull; + +public class PinotConnector + implements Connector +{ + private static final Logger log = Logger.get(PinotConnector.class); + private final LifeCycleManager lifeCycleManager; + private final PinotMetadata metadata; + private final PinotSplitManager splitManager; + private final PinotPageSourceProvider pageSourceProvider; + private final PinotNodePartitioningProvider partitioningProvider; + private final List> sessionProperties; + private final ConnectorPlanOptimizer planOptimizer; + + @Inject + public PinotConnector(LifeCycleManager lifeCycleManager, + PinotMetadata metadata, + PinotSplitManager splitManager, + PinotPageSourceProvider pageSourceProvider, + PinotNodePartitioningProvider partitioningProvider, + PinotSessionProperties pinotSessionProperties, + PinotConnectorPlanOptimizer planOptimizer) + { + this.lifeCycleManager = requireNonNull(lifeCycleManager, "lifeCycleManager is null"); + this.metadata = requireNonNull(metadata, "metadata is null"); + this.splitManager = requireNonNull(splitManager, "splitManager is null"); + this.pageSourceProvider = requireNonNull(pageSourceProvider, "pageSourceProvider is null"); + this.partitioningProvider = requireNonNull(partitioningProvider, "partitioningProvider is null"); + this.sessionProperties = ImmutableList.copyOf(requireNonNull(pinotSessionProperties, "sessionProperties is null").getSessionProperties()); + this.planOptimizer = requireNonNull(planOptimizer, "plan optimizer is null"); + } + + @Override + public ConnectorTransactionHandle beginTransaction(IsolationLevel isolationLevel, boolean readOnly) + { + return INSTANCE; + } + + @Override + public ConnectorMetadata getMetadata(ConnectorTransactionHandle transactionHandle) + { + return metadata; + } + + @Override + public ConnectorSplitManager getSplitManager() + { + return splitManager; + } + + @Override + public ConnectorPageSourceProvider getPageSourceProvider() + { + return pageSourceProvider; + } + + @Override + public ConnectorNodePartitioningProvider getNodePartitioningProvider() + { + return partitioningProvider; + } + + @Override + public List> getSessionProperties() + { + return sessionProperties; + } + + @Override + public ConnectorPlanOptimizerProvider getConnectorPlanOptimizerProvider() + { + return new ConnectorPlanOptimizerProvider() + { + @Override + public Set getLogicalPlanOptimizers() + { + return ImmutableSet.of(planOptimizer); + } + + @Override + public Set getPhysicalPlanOptimizers() + { + return ImmutableSet.of(); + } + }; + } + + @Override + public final void shutdown() + { + try { + lifeCycleManager.stop(); + } + catch (Exception e) { + log.error(e, "Error shutting down connector"); + } + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotConnectorFactory.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotConnectorFactory.java new file mode 100644 index 0000000000000..08348f268c016 --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotConnectorFactory.java @@ -0,0 +1,97 @@ +/* + * 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.pinot; + +import com.facebook.airlift.bootstrap.Bootstrap; +import com.facebook.airlift.json.JsonModule; +import com.facebook.presto.expressions.LogicalRowExpressions; +import com.facebook.presto.spi.ConnectorHandleResolver; +import com.facebook.presto.spi.ConnectorId; +import com.facebook.presto.spi.NodeManager; +import com.facebook.presto.spi.connector.Connector; +import com.facebook.presto.spi.connector.ConnectorContext; +import com.facebook.presto.spi.connector.ConnectorFactory; +import com.facebook.presto.spi.connector.ConnectorNodePartitioningProvider; +import com.facebook.presto.spi.function.FunctionMetadataManager; +import com.facebook.presto.spi.function.StandardFunctionResolution; +import com.facebook.presto.spi.relation.RowExpressionService; +import com.facebook.presto.spi.type.TypeManager; +import com.google.inject.Injector; +import com.google.inject.Scopes; +import org.weakref.jmx.guice.MBeanModule; + +import javax.management.MBeanServer; + +import java.util.Map; + +import static java.lang.management.ManagementFactory.getPlatformMBeanServer; +import static java.util.Objects.requireNonNull; +import static org.weakref.jmx.ObjectNames.generatedNameOf; +import static org.weakref.jmx.guice.ExportBinder.newExporter; + +public class PinotConnectorFactory + implements ConnectorFactory +{ + public PinotConnectorFactory() + { + } + + @Override + public String getName() + { + return "pinot"; + } + + @Override + public ConnectorHandleResolver getHandleResolver() + { + return new PinotHandleResolver(); + } + + @Override + public Connector create(final String connectorId, Map config, ConnectorContext context) + { + requireNonNull(connectorId, "connectorId is null"); + requireNonNull(config, "config is null"); + + try { + Bootstrap app = new Bootstrap( + new JsonModule(), + new MBeanModule(), + new PinotModule(connectorId), binder -> { + binder.bind(MBeanServer.class).toInstance(new RebindSafeMBeanServer(getPlatformMBeanServer())); + binder.bind(ConnectorId.class).toInstance(new ConnectorId(connectorId)); + binder.bind(TypeManager.class).toInstance(context.getTypeManager()); + binder.bind(FunctionMetadataManager.class).toInstance(context.getFunctionMetadataManager()); + binder.bind(NodeManager.class).toInstance(context.getNodeManager()); + binder.bind(RowExpressionService.class).toInstance(context.getRowExpressionService()); + binder.bind(LogicalRowExpressions.class).toInstance(new LogicalRowExpressions(context.getRowExpressionService().getDeterminismEvaluator(), context.getStandardFunctionResolution(), context.getFunctionMetadataManager())); + binder.bind(StandardFunctionResolution.class).toInstance(context.getStandardFunctionResolution()); + binder.bind(PinotMetrics.class).in(Scopes.SINGLETON); + newExporter(binder).export(PinotMetrics.class).as(generatedNameOf(PinotMetrics.class, connectorId)); + binder.bind(ConnectorNodePartitioningProvider.class).to(PinotNodePartitioningProvider.class).in(Scopes.SINGLETON); + }); + + Injector injector = app.strictConfig() + .doNotInitializeLogging() + .setRequiredConfigurationProperties(config) + .initialize(); + + return injector.getInstance(PinotConnector.class); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotConnectorPlanOptimizer.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotConnectorPlanOptimizer.java new file mode 100644 index 0000000000000..47c0211181c5d --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotConnectorPlanOptimizer.java @@ -0,0 +1,234 @@ +/* + * 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.pinot; + +import com.facebook.presto.expressions.LogicalRowExpressions; +import com.facebook.presto.pinot.query.PinotFilterExpressionConverter; +import com.facebook.presto.pinot.query.PinotQueryGenerator; +import com.facebook.presto.pinot.query.PinotQueryGeneratorContext; +import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.ConnectorPlanOptimizer; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.ConnectorTableHandle; +import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.TableHandle; +import com.facebook.presto.spi.VariableAllocator; +import com.facebook.presto.spi.function.FunctionMetadataManager; +import com.facebook.presto.spi.function.StandardFunctionResolution; +import com.facebook.presto.spi.plan.FilterNode; +import com.facebook.presto.spi.plan.PlanNode; +import com.facebook.presto.spi.plan.PlanNodeIdAllocator; +import com.facebook.presto.spi.plan.PlanVisitor; +import com.facebook.presto.spi.plan.TableScanNode; +import com.facebook.presto.spi.relation.RowExpression; +import com.facebook.presto.spi.relation.VariableReferenceExpression; +import com.facebook.presto.spi.type.TypeManager; +import com.google.common.collect.ImmutableList; + +import javax.inject.Inject; + +import java.util.ArrayList; +import java.util.IdentityHashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static com.facebook.presto.pinot.PinotErrorCode.PINOT_UNCLASSIFIED_ERROR; +import static com.facebook.presto.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static com.google.common.collect.ImmutableMap.toImmutableMap; +import static java.util.Objects.requireNonNull; + +public class PinotConnectorPlanOptimizer + implements ConnectorPlanOptimizer +{ + private final PinotQueryGenerator pinotQueryGenerator; + private final TypeManager typeManager; + private final FunctionMetadataManager functionMetadataManager; + private final LogicalRowExpressions logicalRowExpressions; + private final StandardFunctionResolution standardFunctionResolution; + + @Inject + public PinotConnectorPlanOptimizer( + PinotQueryGenerator pinotQueryGenerator, + TypeManager typeManager, + FunctionMetadataManager functionMetadataManager, + LogicalRowExpressions logicalRowExpressions, + StandardFunctionResolution standardFunctionResolution) + { + this.pinotQueryGenerator = requireNonNull(pinotQueryGenerator, "pinot query generator is null"); + this.typeManager = requireNonNull(typeManager, "type manager is null"); + this.functionMetadataManager = requireNonNull(functionMetadataManager, "function manager is null"); + this.logicalRowExpressions = requireNonNull(logicalRowExpressions, "logical row expressions is null"); + this.standardFunctionResolution = requireNonNull(standardFunctionResolution, "standard function resolution is null"); + } + + @Override + public PlanNode optimize(PlanNode maxSubplan, + ConnectorSession session, + VariableAllocator variableAllocator, + PlanNodeIdAllocator idAllocator) + { + Map scanNodes = maxSubplan.accept(new TableFindingVisitor(), null); + TableScanNode pinotTableScanNode = getOnlyPinotTable(scanNodes) + .orElseThrow(() -> new PrestoException(GENERIC_INTERNAL_ERROR, + "Expected to find the pinot table handle for the scan node")); + return maxSubplan.accept(new Visitor(pinotTableScanNode, session, idAllocator), null); + } + + private static Optional getPinotTableHandle(TableScanNode tableScanNode) + { + TableHandle table = tableScanNode.getTable(); + if (table != null) { + ConnectorTableHandle connectorHandle = table.getConnectorHandle(); + if (connectorHandle instanceof PinotTableHandle) { + return Optional.of((PinotTableHandle) connectorHandle); + } + } + return Optional.empty(); + } + + private static Optional getOnlyPinotTable(Map scanNodes) + { + if (scanNodes.size() == 1) { + TableScanNode tableScanNode = scanNodes.keySet().iterator().next(); + if (getPinotTableHandle(tableScanNode).isPresent()) { + return Optional.of(tableScanNode); + } + } + return Optional.empty(); + } + + private static PlanNode replaceChildren(PlanNode node, List children) + { + for (int i = 0; i < node.getSources().size(); i++) { + if (children.get(i) != node.getSources().get(i)) { + return node.replaceChildren(children); + } + } + return node; + } + + private static class TableFindingVisitor + extends PlanVisitor, Void> + { + @Override + public Map visitPlan(PlanNode node, Void context) + { + Map ret = new IdentityHashMap<>(); + node.getSources().forEach(source -> ret.putAll(source.accept(this, context))); + return ret; + } + + @Override + public Map visitTableScan(TableScanNode node, Void context) + { + Map ret = new IdentityHashMap<>(); + ret.put(node, null); + return ret; + } + } + + // Single use visitor that needs the pinot table handle + private class Visitor + extends PlanVisitor + { + private final PlanNodeIdAllocator idAllocator; + private final ConnectorSession session; + private final TableScanNode tableScanNode; + private final IdentityHashMap filtersSplitUp = new IdentityHashMap<>(); + + public Visitor(TableScanNode tableScanNode, ConnectorSession session, PlanNodeIdAllocator idAllocator) + { + this.session = session; + this.idAllocator = idAllocator; + this.tableScanNode = tableScanNode; + // Just making sure that the table exists + getPinotTableHandle(this.tableScanNode).get().getTableName(); + } + + private Optional tryCreatingNewScanNode(PlanNode plan) + { + Optional pql = pinotQueryGenerator.generate(plan, session); + if (!pql.isPresent()) { + return Optional.empty(); + } + PinotTableHandle pinotTableHandle = getPinotTableHandle(tableScanNode).orElseThrow(() -> new PinotException(PINOT_UNCLASSIFIED_ERROR, Optional.empty(), "Expected to find a pinot table handle")); + PinotQueryGeneratorContext context = pql.get().getContext(); + TableHandle oldTableHandle = tableScanNode.getTable(); + LinkedHashMap assignments = context.getAssignments(); + boolean isQueryShort = pql.get().getGeneratedPql().isQueryShort(); + TableHandle newTableHandle = new TableHandle( + oldTableHandle.getConnectorId(), + new PinotTableHandle(pinotTableHandle.getConnectorId(), pinotTableHandle.getSchemaName(), pinotTableHandle.getTableName(), Optional.of(isQueryShort), Optional.of(pql.get().getGeneratedPql())), + oldTableHandle.getTransaction(), + oldTableHandle.getLayout()); + return Optional.of( + new TableScanNode( + idAllocator.getNextId(), + newTableHandle, + ImmutableList.copyOf(assignments.keySet()), + assignments.entrySet().stream().collect(toImmutableMap(Map.Entry::getKey, (e) -> (ColumnHandle) (e.getValue()))), + tableScanNode.getCurrentConstraint(), + tableScanNode.getEnforcedConstraint())); + } + + @Override + public PlanNode visitPlan(PlanNode node, Void context) + { + Optional pushedDownPlan = tryCreatingNewScanNode(node); + return pushedDownPlan.orElseGet(() -> replaceChildren( + node, + node.getSources().stream().map(source -> source.accept(this, null)).collect(toImmutableList()))); + } + + @Override + public PlanNode visitFilter(FilterNode node, Void context) + { + if (filtersSplitUp.containsKey(node)) { + return this.visitPlan(node, context); + } + filtersSplitUp.put(node, null); + FilterNode nodeToRecurseInto = node; + List pushable = new ArrayList<>(); + List nonPushable = new ArrayList<>(); + PinotFilterExpressionConverter pinotFilterExpressionConverter = new PinotFilterExpressionConverter(typeManager, functionMetadataManager, standardFunctionResolution); + for (RowExpression conjunct : LogicalRowExpressions.extractConjuncts(node.getPredicate())) { + try { + conjunct.accept(pinotFilterExpressionConverter, (var) -> new PinotQueryGeneratorContext.Selection(var.getName(), PinotQueryGeneratorContext.Origin.DERIVED)); + pushable.add(conjunct); + } + catch (PinotException pe) { + nonPushable.add(conjunct); + } + } + if (!pushable.isEmpty()) { + FilterNode pushableFilter = new FilterNode(idAllocator.getNextId(), node.getSource(), logicalRowExpressions.combineConjuncts(pushable)); + Optional nonPushableFilter = nonPushable.isEmpty() ? Optional.empty() : Optional.of(new FilterNode(idAllocator.getNextId(), pushableFilter, logicalRowExpressions.combineConjuncts(nonPushable))); + + filtersSplitUp.put(pushableFilter, null); + if (nonPushableFilter.isPresent()) { + FilterNode nonPushableFilterNode = nonPushableFilter.get(); + filtersSplitUp.put(nonPushableFilterNode, null); + nodeToRecurseInto = nonPushableFilterNode; + } + else { + nodeToRecurseInto = pushableFilter; + } + } + return this.visitFilter(nodeToRecurseInto, context); + } + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotErrorCode.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotErrorCode.java new file mode 100644 index 0000000000000..73e6748a4b7d7 --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotErrorCode.java @@ -0,0 +1,70 @@ +/* + * 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.pinot; + +import com.facebook.presto.spi.ErrorCode; +import com.facebook.presto.spi.ErrorCodeSupplier; +import com.facebook.presto.spi.ErrorType; + +import static com.facebook.presto.spi.ErrorType.EXTERNAL; +import static com.facebook.presto.spi.ErrorType.INTERNAL_ERROR; + +public enum PinotErrorCode + implements ErrorCodeSupplier +{ + PINOT_UNSUPPORTED_COLUMN_TYPE(0, EXTERNAL), // schema issues + PINOT_QUERY_GENERATOR_FAILURE(1, INTERNAL_ERROR), // Accepted a query whose pql we couldn't generate + PINOT_INSUFFICIENT_SERVER_RESPONSE(2, EXTERNAL, true), // numServersResponded < numServersQueried + PINOT_EXCEPTION(3, EXTERNAL), // Exception reported by pinot + PINOT_HTTP_ERROR(4, EXTERNAL), // Some non okay http error code + PINOT_UNEXPECTED_RESPONSE(5, EXTERNAL), // Invalid json response with okay http return code + PINOT_UNSUPPORTED_EXPRESSION(6, INTERNAL_ERROR), // Unsupported function + PINOT_UNABLE_TO_FIND_BROKER(7, EXTERNAL), + PINOT_DECODE_ERROR(8, EXTERNAL), + PINOT_INVALID_PQL_GENERATED(9, INTERNAL_ERROR), + PINOT_INVALID_CONFIGURATION(10, INTERNAL_ERROR), + PINOT_UNCLASSIFIED_ERROR(100, EXTERNAL); + + /** + * Connectors can use error codes starting at the range 0x0100_0000 + * See https://github.com/prestodb/presto/wiki/Error-Codes + * + * @see com.facebook.presto.spi.StandardErrorCode + */ + + private final ErrorCode errorCode; + private final boolean retriable; + + PinotErrorCode(int code, ErrorType type, boolean retriable) + { + errorCode = new ErrorCode(code + 0x0505_0000, name(), type); + this.retriable = retriable; + } + + PinotErrorCode(int code, ErrorType type) + { + this(code, type, false); + } + + public boolean isRetriable() + { + return retriable; + } + + @Override + public ErrorCode toErrorCode() + { + return errorCode; + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotException.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotException.java new file mode 100644 index 0000000000000..30c6ce81a4f1c --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotException.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.pinot; + +import com.facebook.presto.spi.PrestoException; + +import java.util.Optional; + +import static java.util.Objects.requireNonNull; + +public class PinotException + extends PrestoException +{ + private final Optional pql; + private final PinotErrorCode pinotErrorCode; + + public PinotException(PinotErrorCode errorCode, Optional pql, String message) + { + this(errorCode, pql, message, null); + } + + public PinotException(PinotErrorCode pinotErrorCode, Optional pql, String message, Throwable throwable) + { + super(requireNonNull(pinotErrorCode, "error code is null"), requireNonNull(message, "message is null"), throwable); + this.pinotErrorCode = pinotErrorCode; + this.pql = requireNonNull(pql, "pql is null"); + } + + public PinotErrorCode getPinotErrorCode() + { + return pinotErrorCode; + } + + @Override + public String getMessage() + { + String message = super.getMessage(); + if (pql.isPresent()) { + message += " with pql \"" + pql.get() + "\""; + } + return message; + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotHandleResolver.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotHandleResolver.java new file mode 100644 index 0000000000000..d4e462b77d01b --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotHandleResolver.java @@ -0,0 +1,55 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.pinot; + +import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.ConnectorHandleResolver; +import com.facebook.presto.spi.ConnectorSplit; +import com.facebook.presto.spi.ConnectorTableHandle; +import com.facebook.presto.spi.ConnectorTableLayoutHandle; +import com.facebook.presto.spi.connector.ConnectorTransactionHandle; + +public class PinotHandleResolver + implements ConnectorHandleResolver +{ + @Override + public Class getTableLayoutHandleClass() + { + return PinotTableLayoutHandle.class; + } + + @Override + public Class getTableHandleClass() + { + return PinotTableHandle.class; + } + + @Override + public Class getColumnHandleClass() + { + return PinotColumnHandle.class; + } + + @Override + public Class getSplitClass() + { + return PinotSplit.class; + } + + @Override + public Class getTransactionHandleClass() + { + return PinotTransactionHandle.class; + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotMetadata.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotMetadata.java new file mode 100644 index 0000000000000..7388818166532 --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotMetadata.java @@ -0,0 +1,181 @@ +/* + * 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.pinot; + +import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.ColumnMetadata; +import com.facebook.presto.spi.ConnectorId; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.ConnectorTableHandle; +import com.facebook.presto.spi.ConnectorTableLayout; +import com.facebook.presto.spi.ConnectorTableLayoutHandle; +import com.facebook.presto.spi.ConnectorTableLayoutResult; +import com.facebook.presto.spi.ConnectorTableMetadata; +import com.facebook.presto.spi.Constraint; +import com.facebook.presto.spi.SchemaTableName; +import com.facebook.presto.spi.SchemaTablePrefix; +import com.facebook.presto.spi.TableNotFoundException; +import com.facebook.presto.spi.connector.ConnectorMetadata; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; + +import javax.inject.Inject; + +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static com.facebook.presto.pinot.PinotColumnHandle.PinotColumnType.REGULAR; +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Locale.ENGLISH; +import static java.util.Objects.requireNonNull; + +public class PinotMetadata + implements ConnectorMetadata +{ + private final String connectorId; + private final PinotConnection pinotPrestoConnection; + + @Inject + public PinotMetadata(ConnectorId connectorId, PinotConnection pinotPrestoConnection) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); + this.pinotPrestoConnection = requireNonNull(pinotPrestoConnection, "pinotPrestoConnection is null"); + } + + @Override + public List listSchemaNames(ConnectorSession session) + { + return ImmutableList.of("default"); + } + + private String getPinotTableNameFromPrestoTableName(String prestoTableName) + { + List allTables = pinotPrestoConnection.getTableNames(); + for (String pinotTableName : allTables) { + if (prestoTableName.equalsIgnoreCase(pinotTableName)) { + return pinotTableName; + } + } + throw new PinotException(PinotErrorCode.PINOT_UNCLASSIFIED_ERROR, Optional.empty(), "Unable to find the presto table " + prestoTableName + " in " + allTables); + } + + @Override + public PinotTableHandle getTableHandle(ConnectorSession session, SchemaTableName tableName) + { + String pinotTableName = getPinotTableNameFromPrestoTableName(tableName.getTableName()); + return new PinotTableHandle(connectorId, tableName.getSchemaName(), pinotTableName); + } + + @Override + public List getTableLayouts( + ConnectorSession session, + ConnectorTableHandle table, + Constraint constraint, + Optional> desiredColumns) + { + // Constraint's don't need to be pushed down since they are already taken care off by the pushdown logic + PinotTableHandle pinotTableHandle = (PinotTableHandle) table; + ConnectorTableLayout layout = new ConnectorTableLayout(new PinotTableLayoutHandle(pinotTableHandle)); + return ImmutableList.of(new ConnectorTableLayoutResult(layout, constraint.getSummary())); + } + + @Override + public ConnectorTableLayout getTableLayout(ConnectorSession session, ConnectorTableLayoutHandle handle) + { + return new ConnectorTableLayout(handle); + } + + @Override + public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle table) + { + PinotTableHandle pinotTableHandle = (PinotTableHandle) table; + checkArgument(pinotTableHandle.getConnectorId().equals(connectorId), "tableHandle is not for this connector"); + SchemaTableName tableName = new SchemaTableName(pinotTableHandle.getSchemaName(), pinotTableHandle.getTableName()); + + return getTableMetadata(tableName); + } + + @Override + public List listTables(ConnectorSession session, String schemaNameOrNull) + { + ImmutableList.Builder builder = ImmutableList.builder(); + for (String table : pinotPrestoConnection.getTableNames()) { + builder.add(new SchemaTableName("default", table)); + } + return builder.build(); + } + + @Override + public Map getColumnHandles(ConnectorSession session, ConnectorTableHandle tableHandle) + { + PinotTableHandle pinotTableHandle = (PinotTableHandle) tableHandle; + checkArgument(pinotTableHandle.getConnectorId().equals(connectorId), "tableHandle is not for this connector"); + + String pinotTableName = getPinotTableNameFromPrestoTableName(pinotTableHandle.getTableName()); + PinotTable table = pinotPrestoConnection.getTable(pinotTableName); + if (table == null) { + throw new TableNotFoundException(pinotTableHandle.toSchemaTableName()); + } + ImmutableMap.Builder columnHandles = ImmutableMap.builder(); + for (ColumnMetadata column : table.getColumnsMetadata()) { + columnHandles.put(column.getName().toLowerCase(ENGLISH), + new PinotColumnHandle(((PinotColumnMetadata) column).getPinotName(), column.getType(), REGULAR)); + } + return columnHandles.build(); + } + + @Override + public Map> listTableColumns(ConnectorSession session, SchemaTablePrefix prefix) + { + requireNonNull(prefix, "prefix is null"); + ImmutableMap.Builder> columns = ImmutableMap.builder(); + for (SchemaTableName tableName : listTables(session, prefix)) { + ConnectorTableMetadata tableMetadata = getTableMetadata(tableName); + // table can disappear during listing operation + if (tableMetadata != null) { + columns.put(tableName, tableMetadata.getColumns()); + } + } + return columns.build(); + } + + private ConnectorTableMetadata getTableMetadata(SchemaTableName tableName) + { + String pinotTableName = getPinotTableNameFromPrestoTableName(tableName.getTableName()); + PinotTable table = pinotPrestoConnection.getTable(pinotTableName); + if (table == null) { + return null; + } + return new ConnectorTableMetadata(tableName, table.getColumnsMetadata()); + } + + private List listTables(ConnectorSession session, SchemaTablePrefix prefix) + { + if (prefix.getSchemaName() == null || prefix.getTableName() == null) { + return listTables(session, prefix.getSchemaName()); + } + return ImmutableList.of(new SchemaTableName(prefix.getSchemaName(), prefix.getTableName())); + } + + @Override + public ColumnMetadata getColumnMetadata( + ConnectorSession session, + ConnectorTableHandle tableHandle, + ColumnHandle columnHandle) + { + return ((PinotColumnHandle) columnHandle).getColumnMetadata(); + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotMetrics.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotMetrics.java new file mode 100644 index 0000000000000..54ded495fb423 --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotMetrics.java @@ -0,0 +1,114 @@ +/* + * 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.pinot; + +import com.facebook.airlift.http.client.Request; +import com.facebook.airlift.http.client.StringResponseHandler.StringResponse; +import org.weakref.jmx.Managed; +import org.weakref.jmx.Nested; + +import javax.annotation.concurrent.ThreadSafe; + +import java.util.concurrent.TimeUnit; + +import static java.util.Locale.ENGLISH; + +@ThreadSafe +public class PinotMetrics +{ + private final PinotMetricsStats getStats = new PinotMetricsStats(false); + private final PinotMetricsStats queryStats = new PinotMetricsStats(true); + private final PinotMetricsStats tablesStats = new PinotMetricsStats(true); + private final PinotMetricsStats schemaStats = new PinotMetricsStats(true); + private final PinotMetricsStats brokerTimeBoundaryStats = new PinotMetricsStats(false); + private final PinotMetricsStats brokerRoutingTableStats = new PinotMetricsStats(true); + + @Managed + @Nested + public PinotMetricsStats getQueryStats() + { + return queryStats; + } + + @Managed + @Nested + public PinotMetricsStats getGetStats() + { + return getStats; + } + + @Managed + @Nested + public PinotMetricsStats getTablesStats() + { + return tablesStats; + } + + @Managed + @Nested + public PinotMetricsStats getSchemaStats() + { + return schemaStats; + } + + @Managed + @Nested + public PinotMetricsStats getBrokerTimeBoundaryStats() + { + return brokerTimeBoundaryStats; + } + + @Managed + @Nested + public PinotMetricsStats getBrokerRoutingTableStats() + { + return brokerRoutingTableStats; + } + + public void monitorRequest( + Request request, + StringResponse response, + long duration, + TimeUnit timeUnit) + { + String[] split = request.getUri().getPath().split("/"); + String secondLast = split.length >= 2 ? split[split.length - 2].toLowerCase(ENGLISH) : null; + String last = split[split.length - 1].toLowerCase(ENGLISH); + if ("post".equalsIgnoreCase(request.getMethod()) && "query".equalsIgnoreCase(last)) { + queryStats.record(response, duration, timeUnit); + } + else if ("get".equalsIgnoreCase(request.getMethod())) { + switch (last) { + case "tables": + tablesStats.record(response, duration, timeUnit); + break; + case "schema": + schemaStats.record(response, duration, timeUnit); + break; + case "debug": + if (secondLast != null) { + switch (secondLast) { + case "routingtable": + brokerRoutingTableStats.record(response, duration, timeUnit); + break; + case "timeboundary": + brokerTimeBoundaryStats.record(response, duration, timeUnit); + break; + } + } + } + getStats.record(response, duration, timeUnit); + } + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotMetricsStats.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotMetricsStats.java new file mode 100644 index 0000000000000..842dc11ee0b1f --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotMetricsStats.java @@ -0,0 +1,85 @@ +/* + * 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.pinot; + +import com.facebook.airlift.http.client.StringResponseHandler.StringResponse; +import com.facebook.airlift.stats.CounterStat; +import com.facebook.airlift.stats.DistributionStat; +import com.facebook.airlift.stats.TimeStat; +import org.weakref.jmx.Managed; +import org.weakref.jmx.Nested; + +import javax.annotation.concurrent.ThreadSafe; + +import java.util.concurrent.TimeUnit; + +import static com.facebook.presto.pinot.PinotUtils.isValidPinotHttpResponseCode; + +@ThreadSafe +public class PinotMetricsStats +{ + private final TimeStat time = new TimeStat(TimeUnit.MILLISECONDS); + private final CounterStat requests = new CounterStat(); + private final CounterStat errorRequests = new CounterStat(); + private DistributionStat responseSize; + + public PinotMetricsStats(boolean withResponse) + { + if (withResponse) { + responseSize = new DistributionStat(); + } + } + + public void record(StringResponse response, long duration, TimeUnit timeUnit) + { + time.add(duration, timeUnit); + requests.update(1); + if (isValidPinotHttpResponseCode(response.getStatusCode())) { + if (responseSize != null) { + responseSize.add(response.getBody().length()); + } + } + else { + errorRequests.update(1); + } + } + + @Managed + @Nested + public TimeStat getTime() + { + return time; + } + + @Managed + @Nested + public CounterStat getRequests() + { + return requests; + } + + @Managed + @Nested + public CounterStat getErrorRequests() + { + return errorRequests; + } + + @Managed + @Nested + public DistributionStat getResponseSize() + { + return responseSize; + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotModule.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotModule.java new file mode 100644 index 0000000000000..c0e09da286302 --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotModule.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.pinot; + +import com.facebook.presto.pinot.query.PinotQueryGenerator; +import com.facebook.presto.spi.type.Type; +import com.facebook.presto.spi.type.TypeManager; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.deser.std.FromStringDeserializer; +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Scopes; +import io.airlift.units.DataSize; +import io.airlift.units.Duration; + +import javax.inject.Inject; + +import java.util.concurrent.Executor; + +import static com.facebook.airlift.concurrent.Threads.threadsNamed; +import static com.facebook.airlift.configuration.ConfigBinder.configBinder; +import static com.facebook.airlift.http.client.HttpClientBinder.httpClientBinder; +import static com.facebook.airlift.json.JsonBinder.jsonBinder; +import static com.facebook.airlift.json.JsonCodec.listJsonCodec; +import static com.facebook.airlift.json.JsonCodecBinder.jsonCodecBinder; +import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature; +import static com.google.common.base.Preconditions.checkArgument; +import static io.airlift.units.DataSize.Unit.MEGABYTE; +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.Executors.newSingleThreadExecutor; +import static java.util.concurrent.TimeUnit.SECONDS; + +/** + * Guice module for the Pinot connector. + */ +public class PinotModule + implements Module +{ + private final String catalogName; + + public PinotModule(String catalogName) + { + this.catalogName = catalogName; + } + + @Override + public void configure(Binder binder) + { + configBinder(binder).bindConfig(PinotConfig.class); + binder.bind(PinotConnector.class).in(Scopes.SINGLETON); + binder.bind(PinotMetadata.class).in(Scopes.SINGLETON); + binder.bind(PinotConnectorPlanOptimizer.class).in(Scopes.SINGLETON); + binder.bind(PinotSplitManager.class).in(Scopes.SINGLETON); + binder.bind(PinotPageSourceProvider.class).in(Scopes.SINGLETON); + binder.bind(PinotClusterInfoFetcher.class).in(Scopes.SINGLETON); + binder.bind(Executor.class).annotatedWith(ForPinot.class) + .toInstance(newSingleThreadExecutor(threadsNamed("pinot-metadata-fetcher-" + catalogName))); + + binder.bind(PinotConnection.class).in(Scopes.SINGLETON); + binder.bind(PinotSessionProperties.class).in(Scopes.SINGLETON); + binder.bind(PinotNodePartitioningProvider.class).in(Scopes.SINGLETON); + binder.bind(PinotQueryGenerator.class).in(Scopes.SINGLETON); + httpClientBinder(binder).bindHttpClient("pinot", ForPinot.class) + .withConfigDefaults(cfg -> { + cfg.setIdleTimeout(new Duration(300, SECONDS)); + cfg.setRequestTimeout(new Duration(300, SECONDS)); + cfg.setMaxConnectionsPerServer(250); + cfg.setMaxContentLength(new DataSize(32, MEGABYTE)); + }); + + jsonBinder(binder).addDeserializerBinding(Type.class).to(TypeDeserializer.class); + jsonCodecBinder(binder).bindMapJsonCodec(String.class, listJsonCodec(PinotTable.class)); + PinotClusterInfoFetcher.addJsonBinders(jsonCodecBinder(binder)); + } + + @SuppressWarnings("serial") + public static final class TypeDeserializer + extends FromStringDeserializer + { + private final TypeManager typeManager; + + @Inject + public TypeDeserializer(TypeManager typeManager) + { + super(Type.class); + this.typeManager = requireNonNull(typeManager, "typeManager is null"); + } + + @Override + protected Type _deserialize(String value, DeserializationContext context) + { + Type type = typeManager.getType(parseTypeSignature(value)); + checkArgument(type != null, "Unknown type %s", value); + return type; + } + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotNodePartitioningProvider.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotNodePartitioningProvider.java new file mode 100644 index 0000000000000..c3995e8ead8f5 --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotNodePartitioningProvider.java @@ -0,0 +1,59 @@ +/* + * 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.pinot; + +import com.facebook.presto.spi.BucketFunction; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.ConnectorSplit; +import com.facebook.presto.spi.connector.ConnectorBucketNodeMap; +import com.facebook.presto.spi.connector.ConnectorNodePartitioningProvider; +import com.facebook.presto.spi.connector.ConnectorPartitioningHandle; +import com.facebook.presto.spi.connector.ConnectorTransactionHandle; +import com.facebook.presto.spi.type.Type; + +import java.util.List; +import java.util.function.ToIntFunction; + +public class PinotNodePartitioningProvider + implements ConnectorNodePartitioningProvider +{ + @Override + public ConnectorBucketNodeMap getBucketNodeMap( + ConnectorTransactionHandle transactionHandle, + ConnectorSession session, + ConnectorPartitioningHandle partitioningHandle) + { + return ConnectorBucketNodeMap.createBucketNodeMap(1); + } + + @Override + public ToIntFunction getSplitBucketFunction( + ConnectorTransactionHandle transactionHandle, + ConnectorSession session, + ConnectorPartitioningHandle partitioningHandle) + { + return value -> 0; + } + + @Override + public BucketFunction getBucketFunction( + ConnectorTransactionHandle transactionHandle, + ConnectorSession session, + ConnectorPartitioningHandle partitioningHandle, + List partitionChannelTypes, + int bucketCount) + { + return null; + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotPageSourceProvider.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotPageSourceProvider.java new file mode 100644 index 0000000000000..1cb063fc7fdb1 --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotPageSourceProvider.java @@ -0,0 +1,100 @@ +/* + * 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.pinot; + +import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.ConnectorId; +import com.facebook.presto.spi.ConnectorPageSource; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.ConnectorSplit; +import com.facebook.presto.spi.ConnectorTableLayoutHandle; +import com.facebook.presto.spi.connector.ConnectorPageSourceProvider; +import com.facebook.presto.spi.connector.ConnectorTransactionHandle; +import com.fasterxml.jackson.databind.ObjectMapper; + +import javax.inject.Inject; + +import java.util.ArrayList; +import java.util.List; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +public class PinotPageSourceProvider + implements ConnectorPageSourceProvider +{ + private final String connectorId; + private final PinotConfig pinotConfig; + private final PinotScatterGatherQueryClient pinotQueryClient; + private final PinotClusterInfoFetcher clusterInfoFetcher; + private final ObjectMapper objectMapper; + + @Inject + public PinotPageSourceProvider( + ConnectorId connectorId, + PinotConfig pinotConfig, + PinotClusterInfoFetcher clusterInfoFetcher, + ObjectMapper objectMapper) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); + this.pinotConfig = requireNonNull(pinotConfig, "pinotConfig is null"); + this.pinotQueryClient = new PinotScatterGatherQueryClient(new PinotScatterGatherQueryClient.Config( + pinotConfig.getIdleTimeout().toMillis(), + pinotConfig.getThreadPoolSize(), + pinotConfig.getMinConnectionsPerServer(), + pinotConfig.getMaxBacklogPerServer(), + pinotConfig.getMaxConnectionsPerServer())); + this.clusterInfoFetcher = requireNonNull(clusterInfoFetcher, "cluster info fetcher is null"); + this.objectMapper = requireNonNull(objectMapper, "object mapper is null"); + } + + @Override + public ConnectorPageSource createPageSource( + ConnectorTransactionHandle transactionHandle, + ConnectorSession session, + ConnectorSplit split, + ConnectorTableLayoutHandle tableLayoutHandle, + List columns) + { + requireNonNull(split, "split is null"); + + PinotSplit pinotSplit = (PinotSplit) split; + checkArgument(pinotSplit.getConnectorId().equals(connectorId), "split is not for this connector"); + + List handles = new ArrayList<>(); + for (ColumnHandle handle : columns) { + handles.add((PinotColumnHandle) handle); + } + + switch (pinotSplit.getSplitType()) { + case SEGMENT: + return new PinotSegmentPageSource( + session, + this.pinotConfig, + this.pinotQueryClient, + pinotSplit, + handles); + case BROKER: + return new PinotBrokerPageSource( + this.pinotConfig, + session, + pinotSplit.getBrokerPql().get(), + handles, + clusterInfoFetcher, + objectMapper); + default: + throw new UnsupportedOperationException("Unknown Pinot split type: " + pinotSplit.getSplitType()); + } + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotPushdownUtils.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotPushdownUtils.java new file mode 100644 index 0000000000000..0cfb3e51470fd --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotPushdownUtils.java @@ -0,0 +1,215 @@ +/* + * 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.pinot; + +import com.facebook.presto.spi.block.SortOrder; +import com.facebook.presto.spi.plan.AggregationNode; +import com.facebook.presto.spi.plan.TopNNode; +import com.facebook.presto.spi.relation.CallExpression; +import com.facebook.presto.spi.relation.ConstantExpression; +import com.facebook.presto.spi.relation.VariableReferenceExpression; +import com.facebook.presto.spi.type.BigintType; +import com.facebook.presto.spi.type.BooleanType; +import com.facebook.presto.spi.type.CharType; +import com.facebook.presto.spi.type.DecimalType; +import com.facebook.presto.spi.type.DoubleType; +import com.facebook.presto.spi.type.IntegerType; +import com.facebook.presto.spi.type.RealType; +import com.facebook.presto.spi.type.SmallintType; +import com.facebook.presto.spi.type.TinyintType; +import com.facebook.presto.spi.type.Type; +import com.facebook.presto.spi.type.VarcharType; +import com.google.common.collect.ImmutableList; +import io.airlift.slice.Slice; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.math.MathContext; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Optional; + +import static com.facebook.presto.pinot.PinotErrorCode.PINOT_UNSUPPORTED_EXPRESSION; +import static com.facebook.presto.spi.type.Decimals.decodeUnscaledValue; +import static com.google.common.base.Preconditions.checkState; +import static java.lang.Float.intBitsToFloat; +import static java.lang.String.format; + +public class PinotPushdownUtils +{ + private PinotPushdownUtils() {} + + public enum ExpressionType + { + GROUP_BY, + AGGREGATE, + } + + /** + * Group by field description + */ + public static class GroupByColumnNode + extends AggregationColumnNode + { + private final VariableReferenceExpression inputColumn; + + public GroupByColumnNode(VariableReferenceExpression inputColumn, VariableReferenceExpression output) + { + super(ExpressionType.GROUP_BY, output); + this.inputColumn = inputColumn; + } + + public VariableReferenceExpression getInputColumn() + { + return inputColumn; + } + + @Override + public String toString() + { + return inputColumn.toString(); + } + } + + /** + * Agg function description. + */ + public static class AggregationFunctionColumnNode + extends AggregationColumnNode + { + private final CallExpression callExpression; + + public AggregationFunctionColumnNode(VariableReferenceExpression output, CallExpression callExpression) + { + super(ExpressionType.AGGREGATE, output); + this.callExpression = callExpression; + } + + public CallExpression getCallExpression() + { + return callExpression; + } + + @Override + public String toString() + { + return callExpression.toString(); + } + } + + public static void checkSupported(boolean condition, String errorMessage, Object... errorMessageArgs) + { + if (!condition) { + throw new PinotException(PinotErrorCode.PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), String.format(errorMessage, errorMessageArgs)); + } + } + + public abstract static class AggregationColumnNode + { + private final ExpressionType expressionType; + private final VariableReferenceExpression outputColumn; + + public AggregationColumnNode(ExpressionType expressionType, VariableReferenceExpression outputColumn) + { + this.expressionType = expressionType; + this.outputColumn = outputColumn; + } + + public VariableReferenceExpression getOutputColumn() + { + return outputColumn; + } + + public ExpressionType getExpressionType() + { + return expressionType; + } + } + + public static List computeAggregationNodes(AggregationNode aggregationNode) + { + int groupByKeyIndex = 0; + ImmutableList.Builder nodeBuilder = ImmutableList.builder(); + for (VariableReferenceExpression outputColumn : aggregationNode.getOutputVariables()) { + AggregationNode.Aggregation agg = aggregationNode.getAggregations().get(outputColumn); + + if (agg != null) { + if (agg.getFilter().isPresent() + || agg.isDistinct() + || agg.getOrderBy().isPresent() + || agg.getMask().isPresent()) { + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), "Unsupported aggregation node " + aggregationNode); + } + nodeBuilder.add(new AggregationFunctionColumnNode(outputColumn, agg.getCall())); + } + else { + // group by output + VariableReferenceExpression inputColumn = aggregationNode.getGroupingKeys().get(groupByKeyIndex); + nodeBuilder.add(new GroupByColumnNode(inputColumn, outputColumn)); + groupByKeyIndex++; + } + } + return nodeBuilder.build(); + } + + public static LinkedHashMap getOrderingScheme(TopNNode topNNode) + { + LinkedHashMap orderingScheme = new LinkedHashMap<>(); + topNNode.getOrderingScheme().getOrderByVariables().forEach(value -> orderingScheme.put(value, topNNode.getOrderingScheme().getOrdering(value))); + return orderingScheme; + } + + private static Number decodeDecimal(BigInteger unscaledValue, DecimalType type) + { + return new BigDecimal(unscaledValue, type.getScale(), new MathContext(type.getPrecision())); + } + + // Copied from com.facebook.presto.sql.planner.LiteralInterpreter.evaluate + public static String getLiteralAsString(ConstantExpression node) + { + Type type = node.getType(); + + if (node.getValue() == null) { + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), String.format("Null constant expression %s with value of type %s", node, type)); + } + if (type instanceof BooleanType) { + return String.valueOf(((Boolean) node.getValue()).booleanValue()); + } + if (type instanceof BigintType || type instanceof TinyintType || type instanceof SmallintType || type instanceof IntegerType) { + Number number = (Number) node.getValue(); + return format("%d", number.longValue()); + } + if (type instanceof DoubleType) { + return node.getValue().toString(); + } + if (type instanceof RealType) { + Long number = (Long) node.getValue(); + return format("%f", intBitsToFloat(number.intValue())); + } + if (type instanceof DecimalType) { + DecimalType decimalType = (DecimalType) type; + if (decimalType.isShort()) { + checkState(node.getValue() instanceof Long); + return decodeDecimal(BigInteger.valueOf((long) node.getValue()), decimalType).toString(); + } + checkState(node.getValue() instanceof Slice); + Slice value = (Slice) node.getValue(); + return decodeDecimal(decodeUnscaledValue(value), decimalType).toString(); + } + if (type instanceof VarcharType || type instanceof CharType) { + return "'" + ((Slice) node.getValue()).toStringUtf8() + "'"; + } + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), String.format("Cannot handle the constant expression %s with value of type %s", node, type)); + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotSegmentPageSource.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotSegmentPageSource.java new file mode 100644 index 0000000000000..bc26d263ecb64 --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotSegmentPageSource.java @@ -0,0 +1,434 @@ +/* + * 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.pinot; + +import com.facebook.presto.pinot.PinotScatterGatherQueryClient.ErrorCode; +import com.facebook.presto.spi.ConnectorPageSource; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.Page; +import com.facebook.presto.spi.PageBuilder; +import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.block.BlockBuilder; +import com.facebook.presto.spi.type.Type; +import com.google.common.collect.ImmutableMap; +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; +import org.apache.pinot.common.data.FieldSpec.DataType; +import org.apache.pinot.common.response.ServerInstance; +import org.apache.pinot.common.utils.DataSchema; +import org.apache.pinot.common.utils.DataTable; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static com.facebook.presto.pinot.PinotErrorCode.PINOT_UNSUPPORTED_COLUMN_TYPE; +import static com.facebook.presto.spi.type.BigintType.BIGINT; +import static com.facebook.presto.spi.type.IntegerType.INTEGER; +import static com.facebook.presto.spi.type.VarcharType.VARCHAR; +import static com.google.common.base.Preconditions.checkArgument; +import static io.airlift.slice.Slices.utf8Slice; +import static java.util.Objects.requireNonNull; + +/** + * This class retrieves Pinot data from a Pinot client, and re-constructs the data into Presto Pages. + */ + +public class PinotSegmentPageSource + implements ConnectorPageSource +{ + private static final Map PINOT_ERROR_CODE_MAP = ImmutableMap.of( + ErrorCode.PINOT_UNCLASSIFIED_ERROR, PinotErrorCode.PINOT_UNCLASSIFIED_ERROR, + ErrorCode.PINOT_INSUFFICIENT_SERVER_RESPONSE, PinotErrorCode.PINOT_INSUFFICIENT_SERVER_RESPONSE, + ErrorCode.PINOT_INVALID_PQL_GENERATED, PinotErrorCode.PINOT_INVALID_PQL_GENERATED); + + private final List columnHandles; + private final PinotConfig pinotConfig; + private final PinotSplit split; + private final PinotScatterGatherQueryClient pinotQueryClient; + private final ConnectorSession session; + + private List columnTypes; + // dataTableList stores the dataTable returned from each server. Each dataTable is constructed to a Page, and then destroyed to save memory. + private LinkedList dataTableList = new LinkedList<>(); + private long completedBytes; + private long readTimeNanos; + private long estimatedMemoryUsageInBytes; + private PinotDataTableWithSize currentDataTable; + private boolean closed; + private boolean isPinotDataFetched; + + public PinotSegmentPageSource( + ConnectorSession session, + PinotConfig pinotConfig, + PinotScatterGatherQueryClient pinotQueryClient, + PinotSplit split, + List columnHandles) + { + this.pinotConfig = requireNonNull(pinotConfig, "pinotConfig is null"); + this.split = requireNonNull(split, "split is null"); + this.pinotQueryClient = requireNonNull(pinotQueryClient, "pinotQueryClient is null"); + this.columnHandles = requireNonNull(columnHandles, "columnHandles is null"); + this.session = requireNonNull(session, "session is null"); + } + + private static void checkExceptions(DataTable dataTable, PinotSplit split) + { + Map metadata = dataTable.getMetadata(); + List exceptions = new ArrayList<>(); + metadata.forEach((k, v) -> { + if (k.startsWith(DataTable.EXCEPTION_METADATA_KEY)) { + exceptions.add(v); + } + }); + if (!exceptions.isEmpty()) { + throw new PinotException(PinotErrorCode.PINOT_EXCEPTION, split.getSegmentPql(), String.format("Encountered %d pinot exceptions for split %s: %s", exceptions.size(), split, exceptions)); + } + } + + @Override + public long getCompletedBytes() + { + return completedBytes; + } + + @Override + public long getReadTimeNanos() + { + return readTimeNanos; + } + + @Override + public long getSystemMemoryUsage() + { + return estimatedMemoryUsageInBytes; + } + + /** + * @return true if is closed or all Pinot data have been processed. + */ + @Override + public boolean isFinished() + { + return closed || (isPinotDataFetched && dataTableList.isEmpty()); + } + + /** + * @return constructed page for pinot data. + */ + @Override + public Page getNextPage() + { + if (isFinished()) { + close(); + return null; + } + if (!isPinotDataFetched) { + fetchPinotData(); + } + // To reduce memory usage, remove dataTable from dataTableList once it's processed. + if (currentDataTable != null) { + estimatedMemoryUsageInBytes -= currentDataTable.getEstimatedSizeInBytes(); + } + if (dataTableList.size() == 0) { + close(); + return null; + } + currentDataTable = dataTableList.pop(); + + PageBuilder pageBuilder = new PageBuilder(columnTypes); + // Note that declared positions in the Page should be the same with number of rows in each Block + pageBuilder.declarePositions(currentDataTable.getDataTable().getNumberOfRows()); + for (int columnHandleIdx = 0; columnHandleIdx < columnHandles.size(); columnHandleIdx++) { + BlockBuilder blockBuilder = pageBuilder.getBlockBuilder(columnHandleIdx); + Type columnType = columnTypes.get(columnHandleIdx); + // Write a block for each column in the original order. + writeBlock(blockBuilder, columnType, columnHandleIdx); + } + + return pageBuilder.build(); + } + + /** + * Fetch data from Pinot for the current split and store the data returned from each Pinot server. + */ + private void fetchPinotData() + { + long startTimeNanos = System.nanoTime(); + try { + Map dataTableMap = queryPinot(session, split); + dataTableMap.values().stream() + // ignore empty tables and tables with 0 rows + .filter(table -> table != null && table.getNumberOfRows() > 0) + .forEach(dataTable -> + { + checkExceptions(dataTable, split); + // Store each dataTable which will later be constructed into Pages. + // Also update estimatedMemoryUsage, mostly represented by the size of all dataTables, using numberOfRows and fieldTypes combined as an estimate + int estimatedTableSizeInBytes = IntStream.rangeClosed(0, dataTable.getDataSchema().size() - 1) + .map(i -> getEstimatedColumnSizeInBytes(dataTable.getDataSchema().getColumnDataType(i)) * dataTable.getNumberOfRows()) + .reduce(0, Integer::sum); + dataTableList.add(new PinotDataTableWithSize(dataTable, estimatedTableSizeInBytes)); + estimatedMemoryUsageInBytes += estimatedTableSizeInBytes; + }); + + this.columnTypes = columnHandles + .stream() + .map(columnHandle -> getTypeForBlock(columnHandle)) + .collect(Collectors.toList()); + isPinotDataFetched = true; + } + finally { + readTimeNanos += System.nanoTime() - startTimeNanos; + } + } + + private Map queryPinot(ConnectorSession session, PinotSplit split) + { + String pql = split.getSegmentPql().orElseThrow(() -> new PinotException(PinotErrorCode.PINOT_INVALID_PQL_GENERATED, Optional.empty(), "Expected the segment split to contain the pql")); + String host = split.getSegmentHost().orElseThrow(() -> new PinotException(PinotErrorCode.PINOT_INVALID_PQL_GENERATED, Optional.empty(), "Expected the segment split to contain the host")); + try { + return ImmutableMap.copyOf( + pinotQueryClient.queryPinotServerForDataTable( + pql, + host, + split.getSegments(), + PinotSessionProperties.getConnectionTimeout(session).toMillis(), + PinotSessionProperties.isIgnoreEmptyResponses(session), + PinotSessionProperties.getPinotRetryCount(session))); + } + catch (PinotScatterGatherQueryClient.PinotException pe) { + throw new PinotException(PINOT_ERROR_CODE_MAP.getOrDefault(pe.getErrorCode(), PinotErrorCode.PINOT_UNCLASSIFIED_ERROR), Optional.of(pql), String.format("Error when hitting host %s", host), pe); + } + } + + @Override + public void close() + { + if (closed) { + return; + } + closed = true; + } + + /** + * Generates the {@link com.facebook.presto.spi.block.Block} for the specific column from the {@link #currentDataTable}. + * + *

Based on the original Pinot column types, write as Presto-supported values to {@link com.facebook.presto.spi.block.BlockBuilder}, e.g. + * FLOAT -> Double, INT -> Long, String -> Slice. + * + * @param blockBuilder blockBuilder for the current column + * @param columnType type of the column + * @param columnIdx column index + */ + + private void writeBlock(BlockBuilder blockBuilder, Type columnType, int columnIdx) + { + Class javaType = columnType.getJavaType(); + DataSchema.ColumnDataType pinotColumnType = currentDataTable.getDataTable().getDataSchema().getColumnDataType(columnIdx); + if (javaType.equals(boolean.class)) { + writeBooleanBlock(blockBuilder, columnType, columnIdx); + } + else if (javaType.equals(long.class)) { + writeLongBlock(blockBuilder, columnType, columnIdx); + } + else if (javaType.equals(double.class)) { + writeDoubleBlock(blockBuilder, columnType, columnIdx); + } + else if (javaType.equals(Slice.class)) { + writeSliceBlock(blockBuilder, columnType, columnIdx); + } + else { + throw new PrestoException( + PINOT_UNSUPPORTED_COLUMN_TYPE, + String.format( + "Failed to write column %s. pinotColumnType %s, javaType %s", + columnHandles.get(columnIdx).getColumnName(), pinotColumnType, javaType)); + } + } + + private void writeBooleanBlock(BlockBuilder blockBuilder, Type columnType, int columnIndex) + { + for (int i = 0; i < currentDataTable.getDataTable().getNumberOfRows(); i++) { + columnType.writeBoolean(blockBuilder, getBoolean(i, columnIndex)); + completedBytes++; + } + } + + private void writeLongBlock(BlockBuilder blockBuilder, Type columnType, int columnIndex) + { + for (int i = 0; i < currentDataTable.getDataTable().getNumberOfRows(); i++) { + columnType.writeLong(blockBuilder, getLong(i, columnIndex)); + completedBytes += Long.BYTES; + } + } + + private void writeDoubleBlock(BlockBuilder blockBuilder, Type columnType, int columnIndex) + { + for (int i = 0; i < currentDataTable.getDataTable().getNumberOfRows(); i++) { + columnType.writeDouble(blockBuilder, getDouble(i, columnIndex)); + completedBytes += Double.BYTES; + } + } + + private void writeSliceBlock(BlockBuilder blockBuilder, Type columnType, int columnIndex) + { + for (int i = 0; i < currentDataTable.getDataTable().getNumberOfRows(); i++) { + Slice slice = getSlice(i, columnIndex); + columnType.writeSlice(blockBuilder, slice, 0, slice.length()); + completedBytes += slice.getBytes().length; + } + } + + Type getType(int columnIndex) + { + checkArgument(columnIndex < columnHandles.size(), "Invalid field index"); + return columnHandles.get(columnIndex).getDataType(); + } + + boolean getBoolean(int rowIdx, int columnIndex) + { + return Boolean.getBoolean(currentDataTable.getDataTable().getString(rowIdx, columnIndex)); + } + + long getLong(int rowIndex, int columnIndex) + { + DataSchema.ColumnDataType dataType = currentDataTable.getDataTable().getDataSchema().getColumnDataType(columnIndex); + // Note columnType in the dataTable could be different from the original columnType in the columnHandle. + // e.g. when original column type is int/long and aggregation value is requested, the returned dataType from Pinot would be double. + // So need to cast it back to the original columnType. + if (dataType.equals(DataType.DOUBLE)) { + return (long) currentDataTable.getDataTable().getDouble(rowIndex, columnIndex); + } + if (dataType.equals(DataType.INT)) { + return (long) currentDataTable.getDataTable().getInt(rowIndex, columnIndex); + } + else { + return currentDataTable.getDataTable().getLong(rowIndex, columnIndex); + } + } + + double getDouble(int rowIndex, int columnIndex) + { + DataSchema.ColumnDataType dataType = currentDataTable.getDataTable().getDataSchema().getColumnDataType(columnIndex); + if (dataType.equals(DataType.FLOAT)) { + return currentDataTable.getDataTable().getFloat(rowIndex, columnIndex); + } + else { + return currentDataTable.getDataTable().getDouble(rowIndex, columnIndex); + } + } + + Slice getSlice(int rowIndex, int columnIndex) + { + checkColumnType(columnIndex, VARCHAR); + DataSchema.ColumnDataType columnType = currentDataTable.getDataTable().getDataSchema().getColumnDataType(columnIndex); + switch (columnType) { + case INT_ARRAY: + int[] intArray = currentDataTable.getDataTable().getIntArray(rowIndex, columnIndex); + return utf8Slice(Arrays.toString(intArray)); + case LONG_ARRAY: + long[] longArray = currentDataTable.getDataTable().getLongArray(rowIndex, columnIndex); + return utf8Slice(Arrays.toString(longArray)); + case FLOAT_ARRAY: + float[] floatArray = currentDataTable.getDataTable().getFloatArray(rowIndex, columnIndex); + return utf8Slice(Arrays.toString(floatArray)); + case DOUBLE_ARRAY: + double[] doubleArray = currentDataTable.getDataTable().getDoubleArray(rowIndex, columnIndex); + return utf8Slice(Arrays.toString(doubleArray)); + case STRING_ARRAY: + String[] stringArray = currentDataTable.getDataTable().getStringArray(rowIndex, columnIndex); + return utf8Slice(Arrays.toString(stringArray)); + case STRING: + String field = currentDataTable.getDataTable().getString(rowIndex, columnIndex); + if (field == null || field.isEmpty()) { + return Slices.EMPTY_SLICE; + } + return Slices.utf8Slice(field); + } + return Slices.EMPTY_SLICE; + } + + /** + * Get estimated size in bytes for the Pinot column. + * Deterministic for numeric fields; use estimate for other types to save calculation. + * + * @param dataType FieldSpec.dataType for Pinot column. + * @return estimated size in bytes. + */ + private int getEstimatedColumnSizeInBytes(DataSchema.ColumnDataType dataType) + { + if (dataType.isNumber()) { + switch (dataType) { + case LONG: + return Long.BYTES; + case FLOAT: + return Float.BYTES; + case DOUBLE: + return Double.BYTES; + case INT: + default: + return Integer.BYTES; + } + } + return pinotConfig.getEstimatedSizeInBytesForNonNumericColumn(); + } + + void checkColumnType(int columnIndex, Type expected) + { + Type actual = getType(columnIndex); + checkArgument(actual.equals(expected), "Expected column %s to be type %s but is %s", columnIndex, expected, actual); + } + + Type getTypeForBlock(PinotColumnHandle pinotColumnHandle) + { + if (pinotColumnHandle.getDataType().equals(INTEGER)) { + return BIGINT; + } + return pinotColumnHandle.getDataType(); + } + + @Override + public long getCompletedPositions() + { + return 0; + } + + private static class PinotDataTableWithSize + { + DataTable dataTable; + int estimatedSizeInBytes; + + PinotDataTableWithSize(DataTable dataTable, int estimatedSizeInBytes) + { + this.dataTable = dataTable; + this.estimatedSizeInBytes = estimatedSizeInBytes; + } + + DataTable getDataTable() + { + return dataTable; + } + + int getEstimatedSizeInBytes() + { + return estimatedSizeInBytes; + } + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotSessionProperties.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotSessionProperties.java new file mode 100644 index 0000000000000..d6bfaff0d49ce --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotSessionProperties.java @@ -0,0 +1,153 @@ +/* + * 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.pinot; + +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.session.PropertyMetadata; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; +import io.airlift.units.Duration; + +import javax.inject.Inject; + +import java.util.List; + +import static com.facebook.presto.spi.session.PropertyMetadata.booleanProperty; +import static com.facebook.presto.spi.session.PropertyMetadata.integerProperty; +import static com.facebook.presto.spi.type.IntegerType.INTEGER; +import static com.facebook.presto.spi.type.VarcharType.createUnboundedVarcharType; +import static com.google.common.base.Preconditions.checkArgument; + +public class PinotSessionProperties +{ + private static final String CONNECTION_TIMEOUT = "connection_timeout"; + private static final String PREFER_BROKER_QUERIES = "prefer_broker_queries"; + private static final String IGNORE_EMPTY_RESPONSES = "ignore_empty_responses"; + private static final String RETRY_COUNT = "retry_count"; + private static final String USE_DATE_TRUNC = "use_date_trunc"; + private static final String NON_AGGREGATE_LIMIT_FOR_BROKER_QUERIES = "non_aggregate_limit_for_broker_queries"; + + @VisibleForTesting + public static final String FORBID_SEGMENT_QUERIES = "forbid_segment_queries"; + + @VisibleForTesting + public static final String NUM_SEGMENTS_PER_SPLIT = "num_segments_per_split"; + + private final List> sessionProperties; + + public static int getNumSegmentsPerSplit(ConnectorSession session) + { + int segmentsPerSplit = session.getProperty(NUM_SEGMENTS_PER_SPLIT, Integer.class); + return segmentsPerSplit <= 0 ? Integer.MAX_VALUE : segmentsPerSplit; + } + + public static boolean isPreferBrokerQueries(ConnectorSession session) + { + return session.getProperty(PREFER_BROKER_QUERIES, Boolean.class); + } + + public static boolean isForbidSegmentQueries(ConnectorSession session) + { + return session.getProperty(FORBID_SEGMENT_QUERIES, Boolean.class); + } + + public static Duration getConnectionTimeout(ConnectorSession session) + { + return session.getProperty(CONNECTION_TIMEOUT, Duration.class); + } + + public static boolean isIgnoreEmptyResponses(ConnectorSession session) + { + return session.getProperty(IGNORE_EMPTY_RESPONSES, Boolean.class); + } + + public static int getPinotRetryCount(ConnectorSession session) + { + return session.getProperty(RETRY_COUNT, Integer.class); + } + + public static boolean isUseDateTruncation(ConnectorSession session) + { + return session.getProperty(USE_DATE_TRUNC, Boolean.class); + } + + public static int getNonAggregateLimitForBrokerQueries(ConnectorSession session) + { + return session.getProperty(NON_AGGREGATE_LIMIT_FOR_BROKER_QUERIES, Integer.class); + } + + @Inject + public PinotSessionProperties(PinotConfig pinotConfig) + { + sessionProperties = ImmutableList.of( + booleanProperty( + PREFER_BROKER_QUERIES, + "Prefer queries to broker even when parallel scan is enabled for aggregation queries", + pinotConfig.isPreferBrokerQueries(), + false), + booleanProperty( + FORBID_SEGMENT_QUERIES, + "Forbid segment queries", + pinotConfig.isForbidSegmentQueries(), + false), + booleanProperty( + IGNORE_EMPTY_RESPONSES, + "Ignore empty or missing pinot server responses", + pinotConfig.isIgnoreEmptyResponses(), + false), + integerProperty( + RETRY_COUNT, + "Retry count for retriable pinot data fetch calls", + pinotConfig.getFetchRetryCount(), + false), + integerProperty( + NON_AGGREGATE_LIMIT_FOR_BROKER_QUERIES, + "Max limit for non aggregate queries to the pinot broker", + pinotConfig.getNonAggregateLimitForBrokerQueries(), + false), + booleanProperty( + USE_DATE_TRUNC, + "Use the new UDF dateTrunc in pinot that is more presto compatible", + pinotConfig.isUseDateTrunc(), + false), + new PropertyMetadata<>( + CONNECTION_TIMEOUT, + "Connection Timeout to talk to Pinot servers", + createUnboundedVarcharType(), + Duration.class, + pinotConfig.getConnectionTimeout(), + false, + value -> Duration.valueOf((String) value), + Duration::toString), + new PropertyMetadata<>( + NUM_SEGMENTS_PER_SPLIT, + "Number of segments of the same host per split", + INTEGER, + Integer.class, + pinotConfig.getNumSegmentsPerSplit(), + false, + value -> { + int ret = ((Number) value).intValue(); + checkArgument(ret > 0, "Number of segments per split must be more than zero"); + return ret; + }, + object -> object)); + } + + public List> getSessionProperties() + { + return sessionProperties; + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotSplit.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotSplit.java new file mode 100644 index 0000000000000..6f02c311cb1e4 --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotSplit.java @@ -0,0 +1,165 @@ +/* + * 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.pinot; + +import com.facebook.presto.pinot.query.PinotQueryGenerator; +import com.facebook.presto.spi.ConnectorSplit; +import com.facebook.presto.spi.HostAddress; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; + +import java.util.List; +import java.util.Optional; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +public class PinotSplit + implements ConnectorSplit +{ + private final String connectorId; + private final SplitType splitType; + + // Properties needed for broker split type + private final Optional brokerPql; + + // Properties needed for segment split type + private final Optional segmentPql; + private final List segments; + private final Optional segmentHost; + + @JsonCreator + public PinotSplit( + @JsonProperty("connectorId") String connectorId, + @JsonProperty("splitType") SplitType splitType, + @JsonProperty("brokerPql") Optional brokerPql, + @JsonProperty("segmentPql") Optional segmentPql, + @JsonProperty("segments") List segments, + @JsonProperty("segmentHost") Optional segmentHost) + { + this.connectorId = requireNonNull(connectorId, "connector id is null"); + this.splitType = requireNonNull(splitType, "splitType id is null"); + this.brokerPql = requireNonNull(brokerPql, "brokerPql is null"); + this.segmentPql = requireNonNull(segmentPql, "table name is null"); + this.segments = ImmutableList.copyOf(requireNonNull(segments, "segment is null")); + this.segmentHost = requireNonNull(segmentHost, "host is null"); + + // make sure the segment properties are present when the split type is segment + if (splitType == SplitType.SEGMENT) { + checkArgument(segmentPql.isPresent(), "Table name is missing from the split"); + checkArgument(!segments.isEmpty(), "Segments are missing from the split"); + checkArgument(segmentHost.isPresent(), "Segment host address is missing from the split"); + } + else { + checkArgument(brokerPql.isPresent(), "brokerPql is missing from the split"); + } + } + + public static PinotSplit createBrokerSplit(String connectorId, PinotQueryGenerator.GeneratedPql brokerPql) + { + return new PinotSplit( + requireNonNull(connectorId, "connector id is null"), + SplitType.BROKER, + Optional.of(requireNonNull(brokerPql, "brokerPql is null")), + Optional.empty(), + ImmutableList.of(), + Optional.empty()); + } + + public static PinotSplit createSegmentSplit(String connectorId, String pql, List segments, String segmentHost) + { + return new PinotSplit( + requireNonNull(connectorId, "connector id is null"), + SplitType.SEGMENT, + Optional.empty(), + Optional.of(requireNonNull(pql, "pql is null")), + requireNonNull(segments, "segments are null"), + Optional.of(requireNonNull(segmentHost, "segmentHost is null"))); + } + + @JsonProperty + public String getConnectorId() + { + return connectorId; + } + + @JsonProperty + public SplitType getSplitType() + { + return splitType; + } + + @JsonProperty + public Optional getBrokerPql() + { + return brokerPql; + } + + @JsonProperty + public Optional getSegmentPql() + { + return segmentPql; + } + + @JsonProperty + public Optional getSegmentHost() + { + return segmentHost; + } + + @JsonProperty + public List getSegments() + { + return segments; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("connectorId", connectorId) + .add("splitType", splitType) + .add("segmentPql", segmentPql) + .add("brokerPql", brokerPql) + .add("segments", segments) + .add("segmentHost", segmentHost) + .toString(); + } + + @Override + public boolean isRemotelyAccessible() + { + return true; + } + + @Override + public List getAddresses() + { + return null; + } + + @Override + public Object getInfo() + { + return this; + } + + public enum SplitType + { + SEGMENT, + BROKER, + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotSplitManager.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotSplitManager.java new file mode 100644 index 0000000000000..34bb63bc83ac2 --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotSplitManager.java @@ -0,0 +1,192 @@ +/* + * 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.pinot; + +import com.facebook.presto.pinot.query.PinotQueryGenerator.GeneratedPql; +import com.facebook.presto.spi.ConnectorId; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.ConnectorSplit; +import com.facebook.presto.spi.ConnectorSplitSource; +import com.facebook.presto.spi.ConnectorTableHandle; +import com.facebook.presto.spi.ConnectorTableLayoutHandle; +import com.facebook.presto.spi.ErrorCode; +import com.facebook.presto.spi.ErrorCodeSupplier; +import com.facebook.presto.spi.ErrorType; +import com.facebook.presto.spi.FixedSplitSource; +import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.connector.ConnectorSplitManager; +import com.facebook.presto.spi.connector.ConnectorTransactionHandle; +import com.google.common.collect.Iterables; + +import javax.inject.Inject; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.Supplier; + +import static com.facebook.presto.pinot.PinotSplit.createBrokerSplit; +import static com.facebook.presto.pinot.PinotSplit.createSegmentSplit; +import static com.facebook.presto.pinot.query.PinotQueryGeneratorContext.TABLE_NAME_SUFFIX_TEMPLATE; +import static com.facebook.presto.pinot.query.PinotQueryGeneratorContext.TIME_BOUNDARY_FILTER_TEMPLATE; +import static com.facebook.presto.spi.ErrorType.USER_ERROR; +import static java.util.Collections.singletonList; +import static java.util.Objects.requireNonNull; + +public class PinotSplitManager + implements ConnectorSplitManager +{ + private final String connectorId; + private final PinotConnection pinotPrestoConnection; + + @Inject + public PinotSplitManager(ConnectorId connectorId, PinotConnection pinotPrestoConnection) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); + this.pinotPrestoConnection = requireNonNull(pinotPrestoConnection, "pinotPrestoConnection is null"); + } + + protected ConnectorSplitSource generateSplitForBrokerBasedScan(GeneratedPql brokerPql) + { + return new FixedSplitSource(singletonList(createBrokerSplit(connectorId, brokerPql))); + } + + protected ConnectorSplitSource generateSplitsForSegmentBasedScan( + PinotTableLayoutHandle pinotLayoutHandle, + ConnectorSession session) + { + PinotTableHandle tableHandle = pinotLayoutHandle.getTable(); + String tableName = tableHandle.getTableName(); + Map>> routingTable; + + routingTable = pinotPrestoConnection.getRoutingTable(tableName); + + List splits = new ArrayList<>(); + if (!routingTable.isEmpty()) { + GeneratedPql segmentPql = tableHandle.getPql().orElseThrow(() -> new PinotException(PinotErrorCode.PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), "Expected to find realtime and offline pql in " + tableHandle)); + PinotClusterInfoFetcher.TimeBoundary timeBoundary = pinotPrestoConnection.getTimeBoundary(tableName); + String realtime = getSegmentPql(segmentPql, "_REALTIME", timeBoundary.getOnlineTimePredicate()); + String offline = getSegmentPql(segmentPql, "_OFFLINE", timeBoundary.getOfflineTimePredicate()); + generateSegmentSplits(splits, routingTable, tableName, "_REALTIME", session, realtime); + generateSegmentSplits(splits, routingTable, tableName, "_OFFLINE", session, offline); + } + + Collections.shuffle(splits); + return new FixedSplitSource(splits); + } + + private String getSegmentPql(GeneratedPql basePql, String suffix, Optional timePredicate) + { + String pql = basePql.getPql().replace(TABLE_NAME_SUFFIX_TEMPLATE, suffix); + if (timePredicate.isPresent()) { + String tp = timePredicate.get(); + pql = pql.replace(TIME_BOUNDARY_FILTER_TEMPLATE, basePql.isHaveFilter() ? tp : " WHERE " + tp); + } + else { + pql = pql.replace(TIME_BOUNDARY_FILTER_TEMPLATE, ""); + } + return pql; + } + + protected void generateSegmentSplits( + List splits, + Map>> routingTable, + String tableName, + String tableNameSuffix, + ConnectorSession session, + String pql) + { + final String finalTableName = tableName + tableNameSuffix; + int segmentsPerSplitConfigured = PinotSessionProperties.getNumSegmentsPerSplit(session); + for (String routingTableName : routingTable.keySet()) { + if (!routingTableName.equalsIgnoreCase(finalTableName)) { + continue; + } + + Map> hostToSegmentsMap = routingTable.get(routingTableName); + hostToSegmentsMap.forEach((host, segments) -> { + int numSegmentsInThisSplit = Math.min(segments.size(), segmentsPerSplitConfigured); + // segments is already shuffled + Iterables.partition(segments, numSegmentsInThisSplit).forEach( + segmentsForThisSplit -> splits.add( + createSegmentSplit(connectorId, pql, segmentsForThisSplit, host))); + }); + } + } + + public enum QueryNotAdequatelyPushedDownErrorCode + implements ErrorCodeSupplier + { + PQL_NOT_PRESENT(1, USER_ERROR, "Query uses unsupported expressions that cannot be pushed into the storage engine. Please see https://XXX for more details"); + + private final ErrorCode errorCode; + + QueryNotAdequatelyPushedDownErrorCode(int code, ErrorType type, String guidance) + { + errorCode = new ErrorCode(code + 0x0625_0000, name() + ": " + guidance, type); + } + + @Override + public ErrorCode toErrorCode() + { + return errorCode; + } + } + + public static class QueryNotAdequatelyPushedDownException + extends PrestoException + { + private final String connectorId; + private final ConnectorTableHandle connectorTableHandle; + + public QueryNotAdequatelyPushedDownException( + QueryNotAdequatelyPushedDownErrorCode errorCode, + ConnectorTableHandle connectorTableHandle, + String connectorId) + { + super(requireNonNull(errorCode, "error code is null"), (String) null); + this.connectorId = requireNonNull(connectorId, "connector id is null"); + this.connectorTableHandle = requireNonNull(connectorTableHandle, "connector table handle is null"); + } + + @Override + public String getMessage() + { + return super.getMessage() + String.format(" table: %s:%s", connectorId, connectorTableHandle); + } + } + + @Override + public ConnectorSplitSource getSplits( + ConnectorTransactionHandle transactionHandle, + ConnectorSession session, + ConnectorTableLayoutHandle layout, + SplitSchedulingContext splitSchedulingContext) + { + PinotTableLayoutHandle pinotLayoutHandle = (PinotTableLayoutHandle) layout; + PinotTableHandle pinotTableHandle = pinotLayoutHandle.getTable(); + Supplier errorSupplier = () -> new QueryNotAdequatelyPushedDownException(QueryNotAdequatelyPushedDownErrorCode.PQL_NOT_PRESENT, pinotTableHandle, connectorId); + if (!pinotTableHandle.getIsQueryShort().orElseThrow(errorSupplier)) { + if (PinotSessionProperties.isForbidSegmentQueries(session)) { + throw errorSupplier.get(); + } + return generateSplitsForSegmentBasedScan(pinotLayoutHandle, session); + } + else { + return generateSplitForBrokerBasedScan(pinotTableHandle.getPql().orElseThrow(errorSupplier)); + } + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotTable.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotTable.java new file mode 100644 index 0000000000000..79fcdddd80b47 --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotTable.java @@ -0,0 +1,62 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.pinot; + +import com.facebook.presto.spi.ColumnMetadata; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; + +import java.util.List; +import java.util.stream.Collectors; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Strings.isNullOrEmpty; +import static java.util.Objects.requireNonNull; + +public class PinotTable +{ + private final String name; + private final List columns; + private final List columnsMetadata; + + @JsonCreator + public PinotTable( + @JsonProperty("name") String name, + @JsonProperty("columns") List columns) + { + checkArgument(!isNullOrEmpty(name), "name is null or is empty"); + this.name = requireNonNull(name, "name is null"); + this.columns = ImmutableList.copyOf(requireNonNull(columns, "columns is null")); + + this.columnsMetadata = columns.stream().map(c -> new PinotColumnMetadata(c.getName(), c.getType())).collect(Collectors.toList()); + } + + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty + public List getColumns() + { + return columns; + } + + public List getColumnsMetadata() + { + return columnsMetadata; + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotTableHandle.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotTableHandle.java new file mode 100644 index 0000000000000..0e7f134608339 --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotTableHandle.java @@ -0,0 +1,129 @@ +/* + * 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.pinot; + +import com.facebook.presto.pinot.query.PinotQueryGenerator; +import com.facebook.presto.spi.ConnectorTableHandle; +import com.facebook.presto.spi.SchemaTableName; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +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 PinotTableHandle + implements ConnectorTableHandle +{ + private final String connectorId; + private final String schemaName; + private final String tableName; + private final Optional isQueryShort; + private final Optional pql; + + public PinotTableHandle( + String connectorId, + String schemaName, + String tableName) + { + this(connectorId, schemaName, tableName, Optional.empty(), Optional.empty()); + } + + @JsonCreator + public PinotTableHandle( + @JsonProperty("connectorId") String connectorId, + @JsonProperty("schemaName") String schemaName, + @JsonProperty("tableName") String tableName, + @JsonProperty("isQueryShort") Optional isQueryShort, + @JsonProperty("pql") Optional pql) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null"); + this.schemaName = requireNonNull(schemaName, "schemaName is null"); + this.tableName = requireNonNull(tableName, "tableName is null"); + this.isQueryShort = requireNonNull(isQueryShort, "safe to execute is null"); + this.pql = requireNonNull(pql, "broker pql is null"); + } + + @JsonProperty + public Optional getPql() + { + return pql; + } + + @JsonProperty + public String getConnectorId() + { + return connectorId; + } + + @JsonProperty + public String getSchemaName() + { + return schemaName; + } + + @JsonProperty + public String getTableName() + { + return tableName; + } + + @JsonProperty + public Optional getIsQueryShort() + { + return isQueryShort; + } + + public SchemaTableName toSchemaTableName() + { + return new SchemaTableName(schemaName, tableName); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PinotTableHandle that = (PinotTableHandle) o; + return Objects.equals(connectorId, that.connectorId) && + Objects.equals(schemaName, that.schemaName) && + Objects.equals(tableName, that.tableName) && + Objects.equals(isQueryShort, that.isQueryShort) && + Objects.equals(pql, that.pql); + } + + @Override + public int hashCode() + { + return Objects.hash(connectorId, schemaName, tableName, isQueryShort, pql); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("connectorId", connectorId) + .add("schemaName", schemaName) + .add("tableName", tableName) + .add("isQueryShort", isQueryShort) + .add("pql", pql) + .toString(); + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotTableLayoutHandle.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotTableLayoutHandle.java new file mode 100644 index 0000000000000..dd234c6b7c185 --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotTableLayoutHandle.java @@ -0,0 +1,66 @@ +/* + * 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.pinot; + +import com.facebook.presto.spi.ConnectorTableLayoutHandle; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +import static java.util.Objects.requireNonNull; + +public class PinotTableLayoutHandle + implements ConnectorTableLayoutHandle +{ + private final PinotTableHandle table; + + @JsonCreator + public PinotTableLayoutHandle( + @JsonProperty("table") PinotTableHandle table) + { + this.table = requireNonNull(table, "table is null"); + } + + @JsonProperty + public PinotTableHandle getTable() + { + return table; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PinotTableLayoutHandle that = (PinotTableLayoutHandle) o; + return Objects.equals(table, that.table); + } + + @Override + public int hashCode() + { + return Objects.hash(table); + } + + @Override + public String toString() + { + return table.toString(); + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotTransactionHandle.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotTransactionHandle.java new file mode 100644 index 0000000000000..5e0fe3f7b31e2 --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotTransactionHandle.java @@ -0,0 +1,22 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.pinot; + +import com.facebook.presto.spi.connector.ConnectorTransactionHandle; + +public enum PinotTransactionHandle + implements ConnectorTransactionHandle +{ + INSTANCE +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotUtils.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotUtils.java new file mode 100644 index 0000000000000..1f13c7e97b478 --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotUtils.java @@ -0,0 +1,53 @@ +/* + * 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.pinot; + +import com.google.common.base.Preconditions; + +import java.util.function.Function; + +import static java.net.HttpURLConnection.HTTP_MULT_CHOICE; +import static java.net.HttpURLConnection.HTTP_OK; + +public class PinotUtils +{ + private PinotUtils() + { + } + + static boolean isValidPinotHttpResponseCode(int status) + { + return status >= HTTP_OK && status < HTTP_MULT_CHOICE; + } + + public static T doWithRetries(int retries, Function caller) + { + PinotException firstError = null; + Preconditions.checkState(retries > 0, "Invalid num of retries %d", retries); + for (int i = 0; i < retries; ++i) { + try { + return caller.apply(i); + } + catch (PinotException e) { + if (firstError == null) { + firstError = e; + } + if (!e.getPinotErrorCode().isRetriable()) { + throw e; + } + } + } + throw firstError; + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/RebindSafeMBeanServer.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/RebindSafeMBeanServer.java new file mode 100644 index 0000000000000..1b7f8b8312660 --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/RebindSafeMBeanServer.java @@ -0,0 +1,344 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * 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.pinot; + +import javax.annotation.concurrent.ThreadSafe; +import javax.management.Attribute; +import javax.management.AttributeList; +import javax.management.AttributeNotFoundException; +import javax.management.InstanceAlreadyExistsException; +import javax.management.InstanceNotFoundException; +import javax.management.IntrospectionException; +import javax.management.InvalidAttributeValueException; +import javax.management.ListenerNotFoundException; +import javax.management.MBeanException; +import javax.management.MBeanInfo; +import javax.management.MBeanRegistrationException; +import javax.management.MBeanServer; +import javax.management.NotCompliantMBeanException; +import javax.management.NotificationFilter; +import javax.management.NotificationListener; +import javax.management.ObjectInstance; +import javax.management.ObjectName; +import javax.management.OperationsException; +import javax.management.QueryExp; +import javax.management.ReflectionException; +import javax.management.loading.ClassLoaderRepository; + +import java.io.ObjectInputStream; +import java.util.Set; + +/** + * MBeanServer wrapper that a ignores calls to registerMBean when there is already + * a MBean registered with the specified object name. + *

+ * This originally existed in hive, raptor and cassandra and I am promoting it to SPI + */ +@ThreadSafe +public class RebindSafeMBeanServer + implements MBeanServer +{ + private final MBeanServer mbeanServer; + + public RebindSafeMBeanServer(MBeanServer mbeanServer) + { + this.mbeanServer = mbeanServer; + } + + /** + * Delegates to the wrapped mbean server, but if a mbean is already registered + * with the specified name, the existing instance is returned. + */ + @Override + public ObjectInstance registerMBean(Object object, ObjectName name) + throws MBeanRegistrationException, NotCompliantMBeanException + { + while (true) { + try { + // try to register the mbean + return mbeanServer.registerMBean(object, name); + } + catch (InstanceAlreadyExistsException ignored) { + } + + try { + // a mbean is already installed, try to return the already registered instance + ObjectInstance objectInstance = mbeanServer.getObjectInstance(name); + return objectInstance; + } + catch (InstanceNotFoundException ignored) { + // the mbean was removed before we could get the reference + // start the whole process over again + } + } + } + + @Override + public void unregisterMBean(ObjectName name) + throws InstanceNotFoundException, MBeanRegistrationException + { + mbeanServer.unregisterMBean(name); + } + + @Override + public ObjectInstance getObjectInstance(ObjectName name) + throws InstanceNotFoundException + { + return mbeanServer.getObjectInstance(name); + } + + @Override + public Set queryMBeans(ObjectName name, QueryExp query) + { + return mbeanServer.queryMBeans(name, query); + } + + @Override + public Set queryNames(ObjectName name, QueryExp query) + { + return mbeanServer.queryNames(name, query); + } + + @Override + public boolean isRegistered(ObjectName name) + { + return mbeanServer.isRegistered(name); + } + + @Override + public Integer getMBeanCount() + { + return mbeanServer.getMBeanCount(); + } + + @Override + public Object getAttribute(ObjectName name, String attribute) + throws MBeanException, AttributeNotFoundException, InstanceNotFoundException, ReflectionException + { + return mbeanServer.getAttribute(name, attribute); + } + + @Override + public AttributeList getAttributes(ObjectName name, String[] attributes) + throws InstanceNotFoundException, ReflectionException + { + return mbeanServer.getAttributes(name, attributes); + } + + @Override + public void setAttribute(ObjectName name, Attribute attribute) + throws InstanceNotFoundException, AttributeNotFoundException, InvalidAttributeValueException, MBeanException, ReflectionException + { + mbeanServer.setAttribute(name, attribute); + } + + @Override + public AttributeList setAttributes(ObjectName name, AttributeList attributes) + throws InstanceNotFoundException, ReflectionException + { + return mbeanServer.setAttributes(name, attributes); + } + + @Override + public Object invoke(ObjectName name, String operationName, Object[] params, String[] signature) + throws InstanceNotFoundException, MBeanException, ReflectionException + { + return mbeanServer.invoke(name, operationName, params, signature); + } + + @Override + public String getDefaultDomain() + { + return mbeanServer.getDefaultDomain(); + } + + @Override + public String[] getDomains() + { + return mbeanServer.getDomains(); + } + + @Override + public void addNotificationListener(ObjectName name, NotificationListener listener, NotificationFilter filter, Object context) + throws InstanceNotFoundException + { + mbeanServer.addNotificationListener(name, listener, filter, context); + } + + @Override + public void addNotificationListener(ObjectName name, ObjectName listener, NotificationFilter filter, Object context) + throws InstanceNotFoundException + { + mbeanServer.addNotificationListener(name, listener, filter, context); + } + + @Override + public void removeNotificationListener(ObjectName name, ObjectName listener) + throws InstanceNotFoundException, ListenerNotFoundException + { + mbeanServer.removeNotificationListener(name, listener); + } + + @Override + public void removeNotificationListener(ObjectName name, ObjectName listener, NotificationFilter filter, Object context) + throws InstanceNotFoundException, ListenerNotFoundException + { + mbeanServer.removeNotificationListener(name, listener, filter, context); + } + + @Override + public void removeNotificationListener(ObjectName name, NotificationListener listener) + throws InstanceNotFoundException, ListenerNotFoundException + { + mbeanServer.removeNotificationListener(name, listener); + } + + @Override + public void removeNotificationListener(ObjectName name, NotificationListener listener, NotificationFilter filter, Object context) + throws InstanceNotFoundException, ListenerNotFoundException + { + mbeanServer.removeNotificationListener(name, listener, filter, context); + } + + @Override + public MBeanInfo getMBeanInfo(ObjectName name) + throws InstanceNotFoundException, IntrospectionException, ReflectionException + { + return mbeanServer.getMBeanInfo(name); + } + + @Override + public boolean isInstanceOf(ObjectName name, String className) + throws InstanceNotFoundException + { + return mbeanServer.isInstanceOf(name, className); + } + + @Override + public Object instantiate(String className) + throws ReflectionException, MBeanException + { + return mbeanServer.instantiate(className); + } + + @Override + public Object instantiate(String className, ObjectName loaderName) + throws ReflectionException, MBeanException, InstanceNotFoundException + { + return mbeanServer.instantiate(className, loaderName); + } + + @Override + public Object instantiate(String className, Object[] params, String[] signature) + throws ReflectionException, MBeanException + { + return mbeanServer.instantiate(className, params, signature); + } + + @Override + public Object instantiate(String className, ObjectName loaderName, Object[] params, String[] signature) + throws ReflectionException, MBeanException, InstanceNotFoundException + { + return mbeanServer.instantiate(className, loaderName, params, signature); + } + + @Override + @Deprecated + @SuppressWarnings("deprecation") + public ObjectInputStream deserialize(ObjectName name, byte[] data) + throws OperationsException + { + return mbeanServer.deserialize(name, data); + } + + @Override + @Deprecated + @SuppressWarnings("deprecation") + public ObjectInputStream deserialize(String className, byte[] data) + throws OperationsException, ReflectionException + { + return mbeanServer.deserialize(className, data); + } + + @Override + @Deprecated + @SuppressWarnings("deprecation") + public ObjectInputStream deserialize(String className, ObjectName loaderName, byte[] data) + throws OperationsException, ReflectionException + { + return mbeanServer.deserialize(className, loaderName, data); + } + + @Override + public ClassLoader getClassLoaderFor(ObjectName mbeanName) + throws InstanceNotFoundException + { + return mbeanServer.getClassLoaderFor(mbeanName); + } + + @Override + public ClassLoader getClassLoader(ObjectName loaderName) + throws InstanceNotFoundException + { + return mbeanServer.getClassLoader(loaderName); + } + + @Override + public ClassLoaderRepository getClassLoaderRepository() + { + return mbeanServer.getClassLoaderRepository(); + } + + @Override + public ObjectInstance createMBean(String className, ObjectName name) + throws ReflectionException, InstanceAlreadyExistsException, MBeanException, NotCompliantMBeanException + { + return mbeanServer.createMBean(className, name); + } + + @Override + public ObjectInstance createMBean(String className, ObjectName name, ObjectName loaderName) + throws ReflectionException, InstanceAlreadyExistsException, MBeanException, NotCompliantMBeanException, InstanceNotFoundException + { + return mbeanServer.createMBean(className, name, loaderName); + } + + @Override + public ObjectInstance createMBean(String className, ObjectName name, Object[] params, String[] signature) + throws ReflectionException, InstanceAlreadyExistsException, MBeanException, NotCompliantMBeanException + { + return mbeanServer.createMBean(className, name, params, signature); + } + + @Override + public ObjectInstance createMBean(String className, ObjectName name, ObjectName loaderName, Object[] params, String[] signature) + throws ReflectionException, InstanceAlreadyExistsException, MBeanException, NotCompliantMBeanException, InstanceNotFoundException + { + return mbeanServer.createMBean(className, name, loaderName, params, signature); + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/query/PinotAggregationProjectConverter.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/query/PinotAggregationProjectConverter.java new file mode 100644 index 0000000000000..308b028ba3553 --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/query/PinotAggregationProjectConverter.java @@ -0,0 +1,262 @@ +/* + * 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.pinot.query; + +import com.facebook.presto.pinot.PinotException; +import com.facebook.presto.pinot.PinotSessionProperties; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.function.FunctionMetadata; +import com.facebook.presto.spi.function.FunctionMetadataManager; +import com.facebook.presto.spi.function.OperatorType; +import com.facebook.presto.spi.function.StandardFunctionResolution; +import com.facebook.presto.spi.relation.CallExpression; +import com.facebook.presto.spi.relation.ConstantExpression; +import com.facebook.presto.spi.relation.RowExpression; +import com.facebook.presto.spi.relation.VariableReferenceExpression; +import com.facebook.presto.spi.type.TypeManager; +import com.google.common.collect.ImmutableMap; +import io.airlift.slice.Slice; +import org.joda.time.DateTimeZone; + +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static com.facebook.presto.pinot.PinotErrorCode.PINOT_UNSUPPORTED_EXPRESSION; +import static com.facebook.presto.pinot.PinotPushdownUtils.getLiteralAsString; +import static com.facebook.presto.pinot.query.PinotExpression.derived; +import static java.lang.String.format; +import static java.util.Locale.ENGLISH; +import static java.util.Objects.requireNonNull; + +public class PinotAggregationProjectConverter + extends PinotProjectExpressionConverter +{ + // Pinot does not support modulus yet + private static final Map PRESTO_TO_PINOT_OPERATORS = ImmutableMap.of( + "-", "SUB", + "+", "ADD", + "*", "MULT", + "/", "DIV"); + private static final String FROM_UNIXTIME = "from_unixtime"; + + private final FunctionMetadataManager functionMetadataManager; + private final ConnectorSession session; + + public PinotAggregationProjectConverter(TypeManager typeManager, FunctionMetadataManager functionMetadataManager, StandardFunctionResolution standardFunctionResolution, ConnectorSession session) + { + super(typeManager, standardFunctionResolution); + this.functionMetadataManager = requireNonNull(functionMetadataManager, "functionMetadataManager is null"); + this.session = requireNonNull(session, "session is null"); + } + + @Override + public PinotExpression visitCall( + CallExpression call, + Map context) + { + Optional basicCallHandlingResult = basicCallHandling(call, context); + if (basicCallHandlingResult.isPresent()) { + return basicCallHandlingResult.get(); + } + FunctionMetadata functionMetadata = functionMetadataManager.getFunctionMetadata(call.getFunctionHandle()); + Optional operatorTypeOptional = functionMetadata.getOperatorType(); + if (operatorTypeOptional.isPresent()) { + OperatorType operatorType = operatorTypeOptional.get(); + if (operatorType.isArithmeticOperator()) { + return handleArithmeticExpression(call, operatorType, context); + } + if (operatorType.isComparisonOperator()) { + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), "Comparison operator not supported: " + call); + } + } + return handleFunction(call, context); + } + + @Override + public PinotExpression visitConstant( + ConstantExpression literal, + Map context) + { + return new PinotExpression(getLiteralAsString(literal), PinotQueryGeneratorContext.Origin.LITERAL); + } + + private PinotExpression handleDateTruncationViaDateTimeConvert( + CallExpression function, + Map context) + { + // Convert SQL standard function `DATE_TRUNC(INTERVAL, DATE/TIMESTAMP COLUMN)` to + // Pinot's equivalent function `dateTimeConvert(columnName, inputFormat, outputFormat, outputGranularity)` + // Pinot doesn't have a DATE/TIMESTAMP type. That means the input column (second argument) has been converted from numeric type to DATE/TIMESTAMP using one of the + // conversion functions in SQL. First step is find the function and find its input column units (seconds, secondsSinceEpoch etc.) + RowExpression timeInputParameter = function.getArguments().get(1); + String inputColumn; + String inputFormat; + + CallExpression timeConversion = getExpressionAsFunction(timeInputParameter, timeInputParameter); + switch (timeConversion.getDisplayName().toLowerCase(ENGLISH)) { + case FROM_UNIXTIME: + inputColumn = timeConversion.getArguments().get(0).accept(this, context).getDefinition(); + inputFormat = "'1:SECONDS:EPOCH'"; + break; + default: + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), "not supported: " + timeConversion.getDisplayName()); + } + + String outputFormat = "'1:MILLISECONDS:EPOCH'"; + String outputGranularity; + + RowExpression intervalParameter = function.getArguments().get(0); + if (!(intervalParameter instanceof ConstantExpression)) { + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), + "interval unit in date_trunc is not supported: " + intervalParameter); + } + + String value = getStringFromConstant(intervalParameter); + switch (value) { + case "second": + outputGranularity = "'1:SECONDS'"; + break; + case "minute": + outputGranularity = "'1:MINUTES'"; + break; + case "hour": + outputGranularity = "'1:HOURS'"; + break; + case "day": + outputGranularity = "'1:DAYS'"; + break; + case "week": + outputGranularity = "'1:WEEKS'"; + break; + case "month": + outputGranularity = "'1:MONTHS'"; + break; + case "quarter": + outputGranularity = "'1:QUARTERS'"; + break; + case "year": + outputGranularity = "'1:YEARS'"; + break; + default: + throw new PinotException( + PINOT_UNSUPPORTED_EXPRESSION, + Optional.empty(), + "interval in date_trunc is not supported: " + value); + } + + return derived("dateTimeConvert(" + inputColumn + ", " + inputFormat + ", " + outputFormat + ", " + outputGranularity + ")"); + } + + private PinotExpression handleDateTruncationViaDateTruncation( + CallExpression function, + Map context) + { + RowExpression timeInputParameter = function.getArguments().get(1); + String inputColumn; + String inputTimeZone; + String inputFormat; + + CallExpression timeConversion = getExpressionAsFunction(timeInputParameter, timeInputParameter); + switch (timeConversion.getDisplayName().toLowerCase(ENGLISH)) { + case FROM_UNIXTIME: + inputColumn = timeConversion.getArguments().get(0).accept(this, context).getDefinition(); + inputTimeZone = timeConversion.getArguments().size() > 1 ? getStringFromConstant(timeConversion.getArguments().get(1)) : DateTimeZone.UTC.getID(); + inputFormat = "seconds"; + break; + default: + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), "not supported: " + timeConversion.getDisplayName()); + } + + RowExpression intervalParameter = function.getArguments().get(0); + if (!(intervalParameter instanceof ConstantExpression)) { + throw new PinotException( + PINOT_UNSUPPORTED_EXPRESSION, + Optional.empty(), + "interval unit in date_trunc is not supported: " + intervalParameter); + } + + return derived("dateTrunc(" + inputColumn + "," + inputFormat + ", " + inputTimeZone + ", " + getStringFromConstant(intervalParameter) + ")"); + } + + private PinotExpression handleArithmeticExpression( + CallExpression expression, + OperatorType operatorType, + Map context) + { + List arguments = expression.getArguments(); + if (arguments.size() == 1) { + String prefix = operatorType == OperatorType.NEGATION ? "-" : ""; + return derived(prefix + arguments.get(0).accept(this, context).getDefinition()); + } + if (arguments.size() == 2) { + PinotExpression left = arguments.get(0).accept(this, context); + PinotExpression right = arguments.get(1).accept(this, context); + String prestoOperator = operatorType.getOperator(); + String pinotOperator = PRESTO_TO_PINOT_OPERATORS.get(prestoOperator); + if (pinotOperator == null) { + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), "Unsupported binary expression " + prestoOperator); + } + return derived(format("%s(%s, %s)", pinotOperator, left.getDefinition(), right.getDefinition())); + } + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), format("Don't know how to interpret %s as an arithmetic expression", expression)); + } + + private PinotExpression handleFunction( + CallExpression function, + Map context) + { + switch (function.getDisplayName().toLowerCase(ENGLISH)) { + case "date_trunc": + boolean useDateTruncation = PinotSessionProperties.isUseDateTruncation(session); + return useDateTruncation ? + handleDateTruncationViaDateTruncation(function, context) : + handleDateTruncationViaDateTimeConvert(function, context); + default: + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), format("function %s not supported yet", function.getDisplayName())); + } + } + + private static String getStringFromConstant(RowExpression expression) + { + if (expression instanceof ConstantExpression) { + Object value = ((ConstantExpression) expression).getValue(); + if (value instanceof String) { + return (String) value; + } + if (value instanceof Slice) { + return ((Slice) value).toStringUtf8(); + } + } + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), "Expected string literal but found " + expression); + } + + private CallExpression getExpressionAsFunction( + RowExpression originalExpression, + RowExpression expression) + { + if (expression instanceof CallExpression) { + CallExpression call = (CallExpression) expression; + if (standardFunctionResolution.isCastFunction(call.getFunctionHandle())) { + if (isImplicitCast(call.getArguments().get(0).getType(), call.getType())) { + return getExpressionAsFunction(originalExpression, call.getArguments().get(0)); + } + } + else { + return call; + } + } + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), "Could not dig function out of expression: " + originalExpression + ", inside of " + expression); + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/query/PinotExpression.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/query/PinotExpression.java new file mode 100644 index 0000000000000..13deee4f76eb3 --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/query/PinotExpression.java @@ -0,0 +1,45 @@ +/* + * 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.pinot.query; + +import com.facebook.presto.pinot.query.PinotQueryGeneratorContext.Origin; + +import static java.util.Objects.requireNonNull; + +public class PinotExpression +{ + private final String definition; + private final Origin origin; + + PinotExpression(String definition, Origin origin) + { + this.definition = requireNonNull(definition, "definition is null"); + this.origin = requireNonNull(origin, "origin is null"); + } + + static PinotExpression derived(String definition) + { + return new PinotExpression(definition, Origin.DERIVED); + } + + public String getDefinition() + { + return definition; + } + + public Origin getOrigin() + { + return origin; + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/query/PinotFilterExpressionConverter.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/query/PinotFilterExpressionConverter.java new file mode 100644 index 0000000000000..736521dc30c87 --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/query/PinotFilterExpressionConverter.java @@ -0,0 +1,229 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.pinot.query; + +import com.facebook.presto.pinot.PinotException; +import com.facebook.presto.pinot.query.PinotQueryGeneratorContext.Origin; +import com.facebook.presto.pinot.query.PinotQueryGeneratorContext.Selection; +import com.facebook.presto.spi.function.FunctionHandle; +import com.facebook.presto.spi.function.FunctionMetadata; +import com.facebook.presto.spi.function.FunctionMetadataManager; +import com.facebook.presto.spi.function.OperatorType; +import com.facebook.presto.spi.function.StandardFunctionResolution; +import com.facebook.presto.spi.relation.CallExpression; +import com.facebook.presto.spi.relation.ConstantExpression; +import com.facebook.presto.spi.relation.InputReferenceExpression; +import com.facebook.presto.spi.relation.LambdaDefinitionExpression; +import com.facebook.presto.spi.relation.RowExpression; +import com.facebook.presto.spi.relation.RowExpressionVisitor; +import com.facebook.presto.spi.relation.SpecialFormExpression; +import com.facebook.presto.spi.relation.VariableReferenceExpression; +import com.facebook.presto.spi.type.Type; +import com.facebook.presto.spi.type.TypeManager; +import com.google.common.collect.ImmutableSet; + +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static com.facebook.presto.pinot.PinotErrorCode.PINOT_UNSUPPORTED_EXPRESSION; +import static com.facebook.presto.pinot.PinotPushdownUtils.getLiteralAsString; +import static com.facebook.presto.pinot.query.PinotExpression.derived; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +/** + * Convert {@link RowExpression} in filter into Pinot complaint expression text + */ +public class PinotFilterExpressionConverter + implements RowExpressionVisitor> +{ + private static final Set LOGICAL_BINARY_OPS_FILTER = ImmutableSet.of("=", "<", "<=", ">", ">=", "<>"); + private final TypeManager typeManager; + private final FunctionMetadataManager functionMetadataManager; + private final StandardFunctionResolution standardFunctionResolution; + + public PinotFilterExpressionConverter( + TypeManager typeManager, + FunctionMetadataManager functionMetadataManager, + StandardFunctionResolution standardFunctionResolution) + { + this.typeManager = requireNonNull(typeManager, "type manager is null"); + this.functionMetadataManager = requireNonNull(functionMetadataManager, "function metadata manager is null"); + this.standardFunctionResolution = requireNonNull(standardFunctionResolution, "standardFunctionResolution is null"); + } + + private PinotExpression handleIn( + SpecialFormExpression specialForm, + boolean isWhitelist, + Function context) + { + return derived(format("(%s %s (%s))", + specialForm.getArguments().get(0).accept(this, context).getDefinition(), + isWhitelist ? "IN" : "NOT IN", + specialForm.getArguments().subList(1, specialForm.getArguments().size()).stream() + .map(argument -> argument.accept(this, context).getDefinition()) + .collect(Collectors.joining(", ")))); + } + + private PinotExpression handleLogicalBinary( + String operator, + CallExpression call, + Function context) + { + if (!LOGICAL_BINARY_OPS_FILTER.contains(operator)) { + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), format("'%s' is not supported in filter", operator)); + } + List arguments = call.getArguments(); + if (arguments.size() == 2) { + return derived(format( + "(%s %s %s)", + arguments.get(0).accept(this, context).getDefinition(), + operator, + arguments.get(1).accept(this, context).getDefinition())); + } + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), format("Unknown logical binary: '%s'", call)); + } + + private PinotExpression handleBetween( + CallExpression between, + Function context) + { + if (between.getArguments().size() == 3) { + RowExpression value = between.getArguments().get(0); + RowExpression min = between.getArguments().get(1); + RowExpression max = between.getArguments().get(2); + + return derived(format( + "(%s BETWEEN %s AND %s)", + value.accept(this, context).getDefinition(), + min.accept(this, context).getDefinition(), + max.accept(this, context).getDefinition())); + } + + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), format("Between operator not supported: %s", between)); + } + + private PinotExpression handleNot(CallExpression not, Function context) + { + if (not.getArguments().size() == 1) { + RowExpression input = not.getArguments().get(0); + if (input instanceof SpecialFormExpression) { + SpecialFormExpression specialFormExpression = (SpecialFormExpression) input; + // NOT operator is only supported on top of the IN expression + if (specialFormExpression.getForm() == SpecialFormExpression.Form.IN) { + return handleIn(specialFormExpression, false, context); + } + } + } + + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), format("NOT operator is supported only on top of IN operator. Received: %s", not)); + } + + private PinotExpression handleCast(CallExpression cast, Function context) + { + if (cast.getArguments().size() == 1) { + RowExpression input = cast.getArguments().get(0); + Type expectedType = cast.getType(); + if (typeManager.canCoerce(input.getType(), expectedType)) { + return input.accept(this, context); + } + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), "Non implicit casts not supported: " + cast); + } + + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), format("This type of CAST operator not supported. Received: %s", cast)); + } + + @Override + public PinotExpression visitCall(CallExpression call, Function context) + { + FunctionHandle functionHandle = call.getFunctionHandle(); + if (standardFunctionResolution.isNotFunction(functionHandle)) { + return handleNot(call, context); + } + if (standardFunctionResolution.isCastFunction(functionHandle)) { + return handleCast(call, context); + } + if (standardFunctionResolution.isBetweenFunction(functionHandle)) { + return handleBetween(call, context); + } + FunctionMetadata functionMetadata = functionMetadataManager.getFunctionMetadata(call.getFunctionHandle()); + Optional operatorTypeOptional = functionMetadata.getOperatorType(); + if (operatorTypeOptional.isPresent()) { + OperatorType operatorType = operatorTypeOptional.get(); + if (operatorType.isArithmeticOperator()) { + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), "Arithmetic expressions are not supported in filter: " + call); + } + if (operatorType.isComparisonOperator()) { + return handleLogicalBinary(operatorType.getOperator(), call, context); + } + } + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), format("function %s not supported in filter", call)); + } + + @Override + public PinotExpression visitInputReference(InputReferenceExpression reference, Function context) + { + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), "Pinot does not support struct dereferencing " + reference); + } + + @Override + public PinotExpression visitConstant(ConstantExpression literal, Function context) + { + return new PinotExpression(getLiteralAsString(literal), Origin.LITERAL); + } + + @Override + public PinotExpression visitLambda(LambdaDefinitionExpression lambda, Function context) + { + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), "Pinot does not support lambda " + lambda); + } + + @Override + public PinotExpression visitVariableReference(VariableReferenceExpression reference, Function context) + { + Selection input = requireNonNull(context.apply(reference), format("Input column %s does not exist in the input: %s", reference, context)); + return new PinotExpression(input.getDefinition(), input.getOrigin()); + } + + @Override + public PinotExpression visitSpecialForm(SpecialFormExpression specialForm, Function context) + { + switch (specialForm.getForm()) { + case IF: + case NULL_IF: + case SWITCH: + case WHEN: + case IS_NULL: + case COALESCE: + case DEREFERENCE: + case ROW_CONSTRUCTOR: + case BIND: + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), "Pinot does not support the special form " + specialForm); + case IN: + return handleIn(specialForm, true, context); + case AND: + case OR: + return derived(format( + "(%s %s %s)", + specialForm.getArguments().get(0).accept(this, context).getDefinition(), + specialForm.getForm().toString(), + specialForm.getArguments().get(1).accept(this, context).getDefinition())); + default: + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), "Unexpected special form: " + specialForm); + } + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/query/PinotProjectExpressionConverter.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/query/PinotProjectExpressionConverter.java new file mode 100644 index 0000000000000..51e0ba8f20679 --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/query/PinotProjectExpressionConverter.java @@ -0,0 +1,137 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.pinot.query; + +import com.facebook.presto.pinot.PinotException; +import com.facebook.presto.pinot.query.PinotQueryGeneratorContext.Selection; +import com.facebook.presto.spi.function.FunctionHandle; +import com.facebook.presto.spi.function.StandardFunctionResolution; +import com.facebook.presto.spi.relation.CallExpression; +import com.facebook.presto.spi.relation.ConstantExpression; +import com.facebook.presto.spi.relation.InputReferenceExpression; +import com.facebook.presto.spi.relation.LambdaDefinitionExpression; +import com.facebook.presto.spi.relation.RowExpression; +import com.facebook.presto.spi.relation.RowExpressionVisitor; +import com.facebook.presto.spi.relation.SpecialFormExpression; +import com.facebook.presto.spi.relation.VariableReferenceExpression; +import com.facebook.presto.spi.type.StandardTypes; +import com.facebook.presto.spi.type.Type; +import com.facebook.presto.spi.type.TypeManager; +import com.google.common.collect.ImmutableSet; + +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static com.facebook.presto.pinot.PinotErrorCode.PINOT_UNSUPPORTED_EXPRESSION; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +class PinotProjectExpressionConverter + implements RowExpressionVisitor> +{ + private static final Set TIME_EQUIVALENT_TYPES = ImmutableSet.of(StandardTypes.BIGINT, StandardTypes.INTEGER, StandardTypes.TINYINT, StandardTypes.SMALLINT); + + protected final TypeManager typeManager; + protected final StandardFunctionResolution standardFunctionResolution; + + public PinotProjectExpressionConverter( + TypeManager typeManager, + StandardFunctionResolution standardFunctionResolution) + { + this.typeManager = requireNonNull(typeManager, "type manager"); + this.standardFunctionResolution = requireNonNull(standardFunctionResolution, "standardFunctionResolution is null"); + } + + @Override + public PinotExpression visitVariableReference( + VariableReferenceExpression reference, + Map context) + { + Selection input = requireNonNull(context.get(reference), format("Input column %s does not exist in the input", reference)); + return new PinotExpression(input.getDefinition(), input.getOrigin()); + } + + @Override + public PinotExpression visitLambda( + LambdaDefinitionExpression lambda, + Map context) + { + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), "Pinot does not support lambda " + lambda); + } + + protected boolean isImplicitCast(Type inputType, Type resultType) + { + if (typeManager.canCoerce(inputType, resultType)) { + return true; + } + return resultType.getTypeSignature().getBase().equals(StandardTypes.TIMESTAMP) && TIME_EQUIVALENT_TYPES.contains(inputType.getTypeSignature().getBase()); + } + + private PinotExpression handleCast( + CallExpression cast, + Map context) + { + if (cast.getArguments().size() == 1) { + RowExpression input = cast.getArguments().get(0); + Type expectedType = cast.getType(); + if (isImplicitCast(input.getType(), expectedType)) { + return input.accept(this, context); + } + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), "Non implicit casts not supported: " + cast); + } + + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), format("This type of CAST operator not supported. Received: %s", cast)); + } + + protected Optional basicCallHandling(CallExpression call, Map context) + { + FunctionHandle functionHandle = call.getFunctionHandle(); + if (standardFunctionResolution.isNotFunction(functionHandle) || standardFunctionResolution.isBetweenFunction(functionHandle)) { + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), "Unsupported function in pinot aggregation: " + functionHandle); + } + if (standardFunctionResolution.isCastFunction(functionHandle)) { + return Optional.of(handleCast(call, context)); + } + return Optional.empty(); + } + + @Override + public PinotExpression visitInputReference( + InputReferenceExpression reference, + Map context) + { + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), "Input reference not supported: " + reference); + } + + @Override + public PinotExpression visitSpecialForm( + SpecialFormExpression specialForm, + Map context) + { + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), "Special form not supported: " + specialForm); + } + + @Override + public PinotExpression visitCall(CallExpression call, Map context) + { + return basicCallHandling(call, context).orElseThrow(() -> new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), "Call not supported: " + call)); + } + + @Override + public PinotExpression visitConstant(ConstantExpression literal, Map context) + { + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), "Constant not supported: " + literal); + } +} diff --git a/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/query/PinotQueryGenerator.java b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/query/PinotQueryGenerator.java new file mode 100644 index 0000000000000..c98e409c323e2 --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/query/PinotQueryGenerator.java @@ -0,0 +1,467 @@ +/* + * 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.pinot.query; + +import com.facebook.airlift.log.Logger; +import com.facebook.presto.pinot.PinotColumnHandle; +import com.facebook.presto.pinot.PinotConfig; +import com.facebook.presto.pinot.PinotException; +import com.facebook.presto.pinot.PinotPushdownUtils.AggregationColumnNode; +import com.facebook.presto.pinot.PinotPushdownUtils.AggregationFunctionColumnNode; +import com.facebook.presto.pinot.PinotPushdownUtils.GroupByColumnNode; +import com.facebook.presto.pinot.PinotSessionProperties; +import com.facebook.presto.pinot.PinotTableHandle; +import com.facebook.presto.pinot.query.PinotQueryGeneratorContext.Selection; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.function.FunctionMetadataManager; +import com.facebook.presto.spi.function.StandardFunctionResolution; +import com.facebook.presto.spi.plan.AggregationNode; +import com.facebook.presto.spi.plan.FilterNode; +import com.facebook.presto.spi.plan.LimitNode; +import com.facebook.presto.spi.plan.PlanNode; +import com.facebook.presto.spi.plan.PlanVisitor; +import com.facebook.presto.spi.plan.ProjectNode; +import com.facebook.presto.spi.plan.TableScanNode; +import com.facebook.presto.spi.plan.TopNNode; +import com.facebook.presto.spi.relation.CallExpression; +import com.facebook.presto.spi.relation.ConstantExpression; +import com.facebook.presto.spi.relation.RowExpression; +import com.facebook.presto.spi.relation.VariableReferenceExpression; +import com.facebook.presto.spi.type.BigintType; +import com.facebook.presto.spi.type.TypeManager; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableMap; + +import javax.inject.Inject; + +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.UUID; +import java.util.stream.Collectors; + +import static com.facebook.presto.pinot.PinotErrorCode.PINOT_UNSUPPORTED_EXPRESSION; +import static com.facebook.presto.pinot.PinotPushdownUtils.checkSupported; +import static com.facebook.presto.pinot.PinotPushdownUtils.computeAggregationNodes; +import static com.facebook.presto.pinot.PinotPushdownUtils.getLiteralAsString; +import static com.facebook.presto.pinot.PinotPushdownUtils.getOrderingScheme; +import static com.facebook.presto.pinot.query.PinotQueryGeneratorContext.Origin.DERIVED; +import static com.facebook.presto.pinot.query.PinotQueryGeneratorContext.Origin.LITERAL; +import static com.facebook.presto.pinot.query.PinotQueryGeneratorContext.Origin.TABLE_COLUMN; +import static com.facebook.presto.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT; +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.lang.String.format; +import static java.util.Locale.ENGLISH; +import static java.util.Objects.requireNonNull; + +public class PinotQueryGenerator +{ + private static final Logger log = Logger.get(PinotQueryGenerator.class); + private static final Map UNARY_AGGREGATION_MAP = ImmutableMap.of( + "min", "min", + "max", "max", + "avg", "avg", + "sum", "sum", + "approx_distinct", "DISTINCTCOUNTHLL"); + + private final PinotConfig pinotConfig; + private final TypeManager typeManager; + private final FunctionMetadataManager functionMetadataManager; + private final StandardFunctionResolution standardFunctionResolution; + private final PinotFilterExpressionConverter pinotFilterExpressionConverter; + private final PinotProjectExpressionConverter pinotProjectExpressionConverter; + + @Inject + public PinotQueryGenerator( + PinotConfig pinotConfig, + TypeManager typeManager, + FunctionMetadataManager functionMetadataManager, + StandardFunctionResolution standardFunctionResolution) + { + this.pinotConfig = requireNonNull(pinotConfig, "pinot config is null"); + this.typeManager = requireNonNull(typeManager, "type manager is null"); + this.functionMetadataManager = requireNonNull(functionMetadataManager, "function metadata manager is null"); + this.standardFunctionResolution = requireNonNull(standardFunctionResolution, "standardFunctionResolution is null"); + this.pinotFilterExpressionConverter = new PinotFilterExpressionConverter(this.typeManager, this.functionMetadataManager, standardFunctionResolution); + this.pinotProjectExpressionConverter = new PinotProjectExpressionConverter(typeManager, standardFunctionResolution); + } + + public static class PinotQueryGeneratorResult + { + private final GeneratedPql generatedPql; + private final PinotQueryGeneratorContext context; + + public PinotQueryGeneratorResult( + GeneratedPql generatedPql, + PinotQueryGeneratorContext context) + { + this.generatedPql = requireNonNull(generatedPql, "generatedPql is null"); + this.context = requireNonNull(context, "context is null"); + } + + public GeneratedPql getGeneratedPql() + { + return generatedPql; + } + + public PinotQueryGeneratorContext getContext() + { + return context; + } + } + + public Optional generate(PlanNode plan, ConnectorSession session) + { + try { + boolean preferBrokerQueries = PinotSessionProperties.isPreferBrokerQueries(session); + PinotQueryGeneratorContext context = requireNonNull(plan.accept(new PinotQueryPlanVisitor(session, preferBrokerQueries), new PinotQueryGeneratorContext()), "Resulting context is null"); + boolean isQueryShort = context.isQueryShort(PinotSessionProperties.getNonAggregateLimitForBrokerQueries(session)); + return Optional.of(new PinotQueryGeneratorResult(context.toQuery(pinotConfig, preferBrokerQueries, isQueryShort), context)); + } + catch (PinotException e) { + log.debug(e, "Possibly benign error when pushing plan into scan node %s", plan); + return Optional.empty(); + } + } + + public static class GeneratedPql + { + final String table; + final String pql; + final List expectedColumnIndices; + final int groupByClauses; + final boolean haveFilter; + final boolean isQueryShort; + + @JsonCreator + public GeneratedPql( + @JsonProperty("table") String table, + @JsonProperty("pql") String pql, + @JsonProperty("expectedColumnIndices") List expectedColumnIndices, + @JsonProperty("groupByClauses") int groupByClauses, + @JsonProperty("haveFilter") boolean haveFilter, + @JsonProperty("isQueryShort") boolean isQueryShort) + { + this.table = table; + this.pql = pql; + this.expectedColumnIndices = expectedColumnIndices; + this.groupByClauses = groupByClauses; + this.haveFilter = haveFilter; + this.isQueryShort = isQueryShort; + } + + @JsonProperty("pql") + public String getPql() + { + return pql; + } + + @JsonProperty("expectedColumnIndices") + public List getExpectedColumnIndices() + { + return expectedColumnIndices; + } + + @JsonProperty("groupByClauses") + public int getGroupByClauses() + { + return groupByClauses; + } + + @JsonProperty("table") + public String getTable() + { + return table; + } + + @JsonProperty("haveFilter") + public boolean isHaveFilter() + { + return haveFilter; + } + + @JsonProperty("isQueryShort") + public boolean isQueryShort() + { + return isQueryShort; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("pql", pql) + .add("table", table) + .add("expectedColumnIndices", expectedColumnIndices) + .add("groupByClauses", groupByClauses) + .add("haveFilter", haveFilter) + .add("isQueryShort", isQueryShort) + .toString(); + } + } + + class PinotQueryPlanVisitor + extends PlanVisitor + { + private final ConnectorSession session; + private final boolean preferBrokerQueries; + + protected PinotQueryPlanVisitor(ConnectorSession session, boolean preferBrokerQueries) + { + this.session = session; + this.preferBrokerQueries = preferBrokerQueries; + } + + @Override + public PinotQueryGeneratorContext visitPlan(PlanNode node, PinotQueryGeneratorContext context) + { + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), "Don't know how to handle plan node of type " + node); + } + + protected VariableReferenceExpression getVariableReference(RowExpression expression) + { + if (expression instanceof VariableReferenceExpression) { + return ((VariableReferenceExpression) expression); + } + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), "Expected a variable reference but got " + expression); + } + + @Override + public PinotQueryGeneratorContext visitFilter(FilterNode node, PinotQueryGeneratorContext context) + { + context = node.getSource().accept(this, context); + requireNonNull(context, "context is null"); + LinkedHashMap selections = context.getSelections(); + String filter = node.getPredicate().accept(pinotFilterExpressionConverter, selections::get).getDefinition(); + return context.withFilter(filter).withOutputColumns(node.getOutputVariables()); + } + + @Override + public PinotQueryGeneratorContext visitProject(ProjectNode node, PinotQueryGeneratorContext contextIn) + { + PinotQueryGeneratorContext context = node.getSource().accept(this, contextIn); + requireNonNull(context, "context is null"); + LinkedHashMap newSelections = new LinkedHashMap<>(); + + node.getOutputVariables().forEach(variable -> { + RowExpression expression = node.getAssignments().get(variable); + PinotExpression pinotExpression = expression.accept( + contextIn.getVariablesInAggregation().contains(variable) ? + new PinotAggregationProjectConverter(typeManager, functionMetadataManager, standardFunctionResolution, session) : pinotProjectExpressionConverter, + context.getSelections()); + newSelections.put( + variable, + new Selection(pinotExpression.getDefinition(), pinotExpression.getOrigin())); + }); + return context.withProject(newSelections); + } + + @Override + public PinotQueryGeneratorContext visitTableScan(TableScanNode node, PinotQueryGeneratorContext contextIn) + { + PinotTableHandle tableHandle = (PinotTableHandle) node.getTable().getConnectorHandle(); + checkSupported(!tableHandle.getPql().isPresent(), "Expect to see no existing pql"); + checkSupported(!tableHandle.getIsQueryShort().isPresent(), "Expect to see no existing pql"); + LinkedHashMap selections = new LinkedHashMap<>(); + node.getOutputVariables().forEach(outputColumn -> { + PinotColumnHandle pinotColumn = (PinotColumnHandle) (node.getAssignments().get(outputColumn)); + checkSupported(pinotColumn.getType().equals(PinotColumnHandle.PinotColumnType.REGULAR), "Unexpected pinot column handle that is not regular: %s", pinotColumn); + selections.put(outputColumn, new Selection(pinotColumn.getColumnName(), TABLE_COLUMN)); + }); + return new PinotQueryGeneratorContext(selections, tableHandle.getTableName()); + } + + private String handleAggregationFunction(CallExpression aggregation, Map inputSelections) + { + String prestoAggregation = aggregation.getDisplayName().toLowerCase(ENGLISH); + List parameters = aggregation.getArguments(); + switch (prestoAggregation) { + case "count": + if (parameters.size() <= 1) { + return format("count(%s)", parameters.isEmpty() ? "*" : inputSelections.get(getVariableReference(parameters.get(0)))); + } + break; + case "approx_percentile": + return handleApproxPercentile(aggregation, inputSelections); + default: + if (UNARY_AGGREGATION_MAP.containsKey(prestoAggregation) && aggregation.getArguments().size() == 1) { + return format("%s(%s)", UNARY_AGGREGATION_MAP.get(prestoAggregation), inputSelections.get(getVariableReference(parameters.get(0)))); + } + } + + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), format("aggregation function '%s' not supported yet", aggregation)); + } + + private String handleApproxPercentile(CallExpression aggregation, Map inputSelections) + { + List inputs = aggregation.getArguments(); + if (inputs.size() != 2) { + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), "Cannot handle approx_percentile function " + aggregation); + } + + String fractionString; + RowExpression fractionInput = inputs.get(1); + + if (fractionInput instanceof ConstantExpression) { + fractionString = getLiteralAsString((ConstantExpression) fractionInput); + } + else if (fractionInput instanceof VariableReferenceExpression) { + Selection fraction = inputSelections.get(fractionInput); + if (fraction.getOrigin() != LITERAL) { + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), + "Cannot handle approx_percentile percentage argument be a non literal " + aggregation); + } + fractionString = fraction.getDefinition(); + } + else { + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), "Expected the fraction to be a constant or a variable " + fractionInput); + } + + int percentile = getValidPercentile(fractionString); + if (percentile < 0) { + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), + format("Cannot handle approx_percentile parsed as %d from input %s (function %s)", percentile, fractionString, aggregation)); + } + return format("PERCENTILEEST%d(%s)", percentile, inputSelections.get(getVariableReference(inputs.get(0)))); + } + + private int getValidPercentile(String fraction) + { + try { + double percent = Double.parseDouble(fraction); + if (percent < 0 || percent > 1) { + throw new PrestoException(INVALID_FUNCTION_ARGUMENT, "Percentile must be between 0 and 1"); + } + percent = percent * 100.0; + if (percent == Math.floor(percent)) { + return (int) percent; + } + } + catch (NumberFormatException ne) { + // Skip + } + return -1; + } + + @Override + public PinotQueryGeneratorContext visitAggregation(AggregationNode node, PinotQueryGeneratorContext contextIn) + { + List aggregationColumnNodes = computeAggregationNodes(node); + + // Make two passes over the aggregatinColumnNodes: In the first pass identify all the variables that will be used + // Then pass that context to the source + // And finally, in the second pass actually generate the PQL + + // 1st pass + Set variablesInAggregation = new HashSet<>(); + for (AggregationColumnNode expression : aggregationColumnNodes) { + switch (expression.getExpressionType()) { + case GROUP_BY: { + GroupByColumnNode groupByColumn = (GroupByColumnNode) expression; + VariableReferenceExpression groupByInputColumn = getVariableReference(groupByColumn.getInputColumn()); + variablesInAggregation.add(groupByInputColumn); + break; + } + case AGGREGATE: { + AggregationFunctionColumnNode aggregationNode = (AggregationFunctionColumnNode) expression; + variablesInAggregation.addAll( + aggregationNode + .getCallExpression() + .getArguments() + .stream() + .filter(argument -> argument instanceof VariableReferenceExpression) + .map(argument -> (VariableReferenceExpression) argument) + .collect(Collectors.toList())); + break; + } + default: + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), "unknown aggregation expression: " + expression.getExpressionType()); + } + } + + // now visit the child project node + PinotQueryGeneratorContext context = node.getSource().accept(this, contextIn.withVariablesInAggregation(variablesInAggregation)); + requireNonNull(context, "context is null"); + checkSupported(!node.getStep().isOutputPartial(), "partial aggregations are not supported in Pinot pushdown framework"); + checkSupported(preferBrokerQueries, "Cannot push aggregation in segment mode"); + + // 2nd pass + LinkedHashMap newSelections = new LinkedHashMap<>(); + LinkedHashSet groupByColumns = new LinkedHashSet<>(); + Set hiddenColumnSet = new HashSet<>(context.getHiddenColumnSet()); + int aggregations = 0; + boolean groupByExists = false; + + for (AggregationColumnNode expression : aggregationColumnNodes) { + switch (expression.getExpressionType()) { + case GROUP_BY: { + GroupByColumnNode groupByColumn = (GroupByColumnNode) expression; + VariableReferenceExpression groupByInputColumn = getVariableReference(groupByColumn.getInputColumn()); + VariableReferenceExpression outputColumn = getVariableReference(groupByColumn.getOutputColumn()); + Selection pinotColumn = requireNonNull(context.getSelections().get(groupByInputColumn), "Group By column " + groupByInputColumn + " doesn't exist in input " + context.getSelections()); + + newSelections.put(outputColumn, new Selection(pinotColumn.getDefinition(), pinotColumn.getOrigin())); + groupByColumns.add(outputColumn); + groupByExists = true; + break; + } + case AGGREGATE: { + AggregationFunctionColumnNode aggregationNode = (AggregationFunctionColumnNode) expression; + String pinotAggFunction = handleAggregationFunction(aggregationNode.getCallExpression(), context.getSelections()); + newSelections.put(getVariableReference(aggregationNode.getOutputColumn()), new Selection(pinotAggFunction, DERIVED)); + aggregations++; + break; + } + default: + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.empty(), "unknown aggregation expression: " + expression.getExpressionType()); + } + } + + // Handling non-aggregated group by + if (groupByExists && aggregations == 0) { + VariableReferenceExpression hidden = new VariableReferenceExpression(UUID.randomUUID().toString(), BigintType.BIGINT); + newSelections.put(hidden, new Selection("count(*)", DERIVED)); + hiddenColumnSet.add(hidden); + aggregations++; + } + return context.withAggregation(newSelections, groupByColumns, aggregations, hiddenColumnSet); + } + + @Override + public PinotQueryGeneratorContext visitLimit(LimitNode node, PinotQueryGeneratorContext context) + { + checkSupported(!node.isPartial(), String.format("pinot query generator cannot handle partial limit")); + checkSupported(preferBrokerQueries, "Cannot push limit in segment mode"); + context = node.getSource().accept(this, context); + requireNonNull(context, "context is null"); + return context.withLimit(node.getCount()).withOutputColumns(node.getOutputVariables()); + } + + @Override + public PinotQueryGeneratorContext visitTopN(TopNNode node, PinotQueryGeneratorContext context) + { + context = node.getSource().accept(this, context); + requireNonNull(context, "context is null"); + checkSupported(preferBrokerQueries, "Cannot push topn in segment mode"); + checkSupported(node.getStep().equals(TopNNode.Step.SINGLE), "Can only push single logical topn in"); + return context.withTopN(getOrderingScheme(node), node.getCount()).withOutputColumns(node.getOutputVariables()); + } + } +} 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 new file mode 100644 index 0000000000000..e7388f3a3aea8 --- /dev/null +++ b/presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/query/PinotQueryGeneratorContext.java @@ -0,0 +1,468 @@ +/* + * 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.pinot.query; + +import com.facebook.presto.pinot.PinotColumnHandle; +import com.facebook.presto.pinot.PinotConfig; +import com.facebook.presto.pinot.PinotException; +import com.facebook.presto.spi.block.SortOrder; +import com.facebook.presto.spi.relation.VariableReferenceExpression; +import com.google.common.base.Joiner; +import com.google.common.collect.ImmutableList; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.OptionalInt; +import java.util.Set; +import java.util.stream.Collectors; + +import static com.facebook.presto.pinot.PinotErrorCode.PINOT_QUERY_GENERATOR_FAILURE; +import static com.facebook.presto.pinot.PinotErrorCode.PINOT_UNSUPPORTED_EXPRESSION; +import static com.facebook.presto.pinot.PinotPushdownUtils.checkSupported; +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.lang.StrictMath.toIntExact; +import static java.lang.String.format; +import static java.util.Locale.ENGLISH; +import static java.util.Objects.requireNonNull; + +/** + * Encapsulates the components needed to construct a PQL query and provides methods to update the current context with new operations. + */ +public class PinotQueryGeneratorContext +{ + public static final String TIME_BOUNDARY_FILTER_TEMPLATE = "__TIME_BOUNDARY_FILTER_TEMPLATE__"; + public static final String TABLE_NAME_SUFFIX_TEMPLATE = "__TABLE_NAME_SUFFIX_TEMPLATE__"; + // Fields defining the query + // order map that maps the column definition in terms of input relation column(s) + private final LinkedHashMap selections; + private final LinkedHashSet groupByColumns; + private final LinkedHashMap topNColumnOrderingMap; + private final Set hiddenColumnSet; + private final Set variablesInAggregation; + private final Optional from; + private final Optional filter; + private final OptionalInt limit; + private final int aggregations; + + public boolean isQueryShort(int nonAggregateRowLimit) + { + return hasAggregation() || limit.orElse(Integer.MAX_VALUE) < nonAggregateRowLimit; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("selections", selections) + .add("groupByColumns", groupByColumns) + .add("hiddenColumnSet", hiddenColumnSet) + .add("from", from) + .add("filter", filter) + .add("limit", limit) + .add("aggregations", aggregations) + .toString(); + } + + PinotQueryGeneratorContext() + { + this(new LinkedHashMap<>(), null); + } + + PinotQueryGeneratorContext( + LinkedHashMap selections, + String from) + { + this( + selections, + Optional.ofNullable(from), + Optional.empty(), + 0, + new LinkedHashSet<>(), + new LinkedHashMap<>(), + OptionalInt.empty(), + new HashSet<>(), + new HashSet<>()); + } + + private PinotQueryGeneratorContext( + LinkedHashMap selections, + Optional from, + Optional filter, + int aggregations, + LinkedHashSet groupByColumns, + LinkedHashMap topNColumnOrderingMap, + OptionalInt limit, + Set variablesInAggregation, + Set hiddenColumnSet) + { + this.selections = new LinkedHashMap<>(requireNonNull(selections, "selections can't be null")); + this.from = requireNonNull(from, "source can't be null"); + this.aggregations = aggregations; + this.groupByColumns = new LinkedHashSet<>(requireNonNull(groupByColumns, "groupByColumns can't be null. It could be empty if not available")); + this.topNColumnOrderingMap = new LinkedHashMap<>(requireNonNull(topNColumnOrderingMap, "topNColumnOrderingMap can't be null. It could be empty if not available")); + this.filter = requireNonNull(filter, "filter is null"); + this.limit = requireNonNull(limit, "limit is null"); + this.hiddenColumnSet = requireNonNull(hiddenColumnSet, "hidden column set is null"); + this.variablesInAggregation = requireNonNull(variablesInAggregation, "variables in aggregation is null"); + } + + /** + * Apply the given filter to current context and return the updated context. Throws error for invalid operations. + */ + public PinotQueryGeneratorContext withFilter(String filter) + { + checkSupported(!hasFilter(), "There already exists a filter. Pinot doesn't support filters at multiple levels"); + checkSupported(!hasAggregation(), "Pinot doesn't support filtering the results of aggregation"); + checkSupported(!hasLimit(), "Pinot doesn't support filtering on top of the limit"); + return new PinotQueryGeneratorContext( + selections, + from, + Optional.of(filter), + aggregations, + groupByColumns, + topNColumnOrderingMap, + limit, + variablesInAggregation, + hiddenColumnSet); + } + + /** + * Apply the aggregation to current context and return the updated context. Throws error for invalid operations. + */ + public PinotQueryGeneratorContext withAggregation( + LinkedHashMap newSelections, + LinkedHashSet groupByColumns, + int aggregations, + Set hiddenColumnSet) + { + // there is only one aggregation supported. + checkSupported(!hasAggregation(), "Pinot doesn't support aggregation on top of the aggregated data"); + checkSupported(!hasLimit(), "Pinot doesn't support aggregation on top of the limit"); + checkSupported(aggregations > 0, "Invalid number of aggregations"); + return new PinotQueryGeneratorContext(newSelections, from, filter, aggregations, groupByColumns, topNColumnOrderingMap, limit, variablesInAggregation, hiddenColumnSet); + } + + /** + * Apply new selections/project to current context and return the updated context. Throws error for invalid operations. + */ + public PinotQueryGeneratorContext withProject(LinkedHashMap newSelections) + { + checkSupported(groupByColumns.isEmpty(), "Pinot doesn't yet support new selections on top of the grouped by data"); + return new PinotQueryGeneratorContext( + newSelections, + from, + filter, + aggregations, + groupByColumns, + topNColumnOrderingMap, + limit, + variablesInAggregation, + hiddenColumnSet); + } + + private static int checkForValidLimit(long limit) + { + if (limit <= 0 || limit > Integer.MAX_VALUE) { + throw new PinotException(PINOT_QUERY_GENERATOR_FAILURE, Optional.empty(), "Limit " + limit + " not supported: Limit is not being pushed down"); + } + return toIntExact(limit); + } + + /** + * Apply limit to current context and return the updated context. Throws error for invalid operations. + */ + public PinotQueryGeneratorContext withLimit(long limit) + { + int intLimit = checkForValidLimit(limit); + checkSupported(!hasLimit(), "Limit already exists. Pinot doesn't support limit on top of another limit"); + return new PinotQueryGeneratorContext( + selections, + from, + filter, + aggregations, + groupByColumns, + topNColumnOrderingMap, + OptionalInt.of(intLimit), + variablesInAggregation, + hiddenColumnSet); + } + + /** + * Apply order by to current context and return the updated context. Throws error for invalid operations. + */ + public PinotQueryGeneratorContext withTopN(LinkedHashMap orderByColumnOrderingMap, long limit) + { + checkSupported(!hasLimit(), "Limit already exists. Pinot doesn't support order by limit on top of another limit"); + checkSupported(!hasAggregation(), "Pinot doesn't support ordering on top of the aggregated data"); + int intLimit = checkForValidLimit(limit); + return new PinotQueryGeneratorContext( + selections, + from, + filter, + aggregations, + groupByColumns, + orderByColumnOrderingMap, + OptionalInt.of(intLimit), + variablesInAggregation, + hiddenColumnSet); + } + + private boolean hasFilter() + { + return filter.isPresent(); + } + + private boolean hasLimit() + { + return limit.isPresent(); + } + + private boolean hasAggregation() + { + return aggregations > 0; + } + + private boolean hasOrderBy() + { + return !topNColumnOrderingMap.isEmpty(); + } + + public LinkedHashMap getSelections() + { + return selections; + } + + public Set getHiddenColumnSet() + { + return hiddenColumnSet; + } + + Set getVariablesInAggregation() + { + return variablesInAggregation; + } + + /** + * Convert the current context to a PQL + */ + public PinotQueryGenerator.GeneratedPql toQuery(PinotConfig pinotConfig, boolean preferBrokerQueries, boolean isQueryShort) + { + boolean forBroker = preferBrokerQueries && isQueryShort; + if (!pinotConfig.isAllowMultipleAggregations() && aggregations > 1 && !groupByColumns.isEmpty()) { + throw new PinotException(PINOT_QUERY_GENERATOR_FAILURE, Optional.empty(), "Multiple aggregates in the presence of group by is forbidden"); + } + + if (hasLimit() && aggregations > 1 && !groupByColumns.isEmpty()) { + throw new PinotException(PINOT_QUERY_GENERATOR_FAILURE, Optional.empty(), "Multiple aggregates in the presence of group by and limit is forbidden"); + } + + String expressions = selections.entrySet().stream() + .filter(s -> !groupByColumns.contains(s.getKey())) // remove the group by columns from the query as Pinot barfs if the group by column is an expression + .map(s -> s.getValue().getDefinition()) + .collect(Collectors.joining(", ")); + + String tableName = from.orElseThrow(() -> new PinotException(PINOT_QUERY_GENERATOR_FAILURE, Optional.empty(), "Table name not encountered yet")); + String query = "SELECT " + expressions + " FROM " + tableName + (forBroker ? "" : TABLE_NAME_SUFFIX_TEMPLATE); + if (filter.isPresent()) { + String filterString = filter.get(); + // this is hack!!!. Ideally we want to clone the scan pipeline and create/update the filter in the scan pipeline to contain this filter and + // at the same time add the time column to scan so that the query generator doesn't fail when it looks up the time column in scan output columns + query += format(" WHERE %s%s", filterString, forBroker ? "" : TIME_BOUNDARY_FILTER_TEMPLATE); + } + else if (!forBroker) { + query += TIME_BOUNDARY_FILTER_TEMPLATE; + } + + if (!groupByColumns.isEmpty()) { + String groupByExpr = groupByColumns.stream().map(x -> selections.get(x).getDefinition()).collect(Collectors.joining(", ")); + query = query + " GROUP BY " + groupByExpr; + } + + if (hasOrderBy()) { + String orderByExpressions = topNColumnOrderingMap.entrySet().stream().map(entry -> selections.get(entry.getKey()).getDefinition() + (entry.getValue().isAscending() ? "" : " DESC")).collect(Collectors.joining(", ")); + query = query + " ORDER BY " + orderByExpressions; + } + // Rules for limit: + // - If its a selection query: + // + given limit or configured limit + // - Else if has group by: + // + ensure that only one aggregation + // + default limit or configured top limit + // - Fail if limit is invalid + + String limitKeyWord = ""; + int queryLimit = -1; + + if (!hasAggregation()) { + if (!limit.isPresent() && forBroker) { + throw new PinotException(PINOT_QUERY_GENERATOR_FAILURE, Optional.empty(), "Broker non aggregate queries have to have a limit"); + } + else { + queryLimit = limit.orElseGet(pinotConfig::getLimitLargeForSegment); + } + limitKeyWord = "LIMIT"; + } + else if (!groupByColumns.isEmpty()) { + limitKeyWord = "TOP"; + if (limit.isPresent()) { + if (aggregations > 1) { + throw new PinotException(PINOT_QUERY_GENERATOR_FAILURE, Optional.of(query), + "Pinot has weird semantics with group by and multiple aggregation functions and limits"); + } + else { + queryLimit = limit.getAsInt(); + } + } + else { + queryLimit = pinotConfig.getTopNLarge(); + } + } + + if (!limitKeyWord.isEmpty()) { + query += " " + limitKeyWord + " " + queryLimit; + } + + List columnHandles = ImmutableList.copyOf(getAssignments().values()); + return new PinotQueryGenerator.GeneratedPql(tableName, query, getIndicesMappingFromPinotSchemaToPrestoSchema(query, columnHandles), groupByColumns.size(), filter.isPresent(), isQueryShort); + } + + private List getIndicesMappingFromPinotSchemaToPrestoSchema(String query, List handles) + { + LinkedHashMap expressionsInPinotOrder = new LinkedHashMap<>(); + for (VariableReferenceExpression groupByColumn : groupByColumns) { + Selection groupByColumnDefinition = selections.get(groupByColumn); + if (groupByColumnDefinition == null) { + throw new IllegalStateException(format( + "Group By column (%s) definition not found in input selections: %s", + groupByColumn, + Joiner.on(",").withKeyValueSeparator(":").join(selections))); + } + expressionsInPinotOrder.put(groupByColumn, groupByColumnDefinition); + } + expressionsInPinotOrder.putAll(selections); + + checkSupported( + handles.size() == expressionsInPinotOrder.keySet().stream().filter(key -> !hiddenColumnSet.contains(key)).count(), + "Expected returned expressions %s to match selections %s", + Joiner.on(",").withKeyValueSeparator(":").join(expressionsInPinotOrder), Joiner.on(",").join(handles)); + + Map nameToIndex = new HashMap<>(); + for (int i = 0; i < handles.size(); i++) { + PinotColumnHandle columnHandle = handles.get(i); + VariableReferenceExpression columnName = new VariableReferenceExpression(columnHandle.getColumnName().toLowerCase(ENGLISH), columnHandle.getDataType()); + Integer previous = nameToIndex.put(columnName, i); + if (previous != null) { + throw new PinotException(PINOT_UNSUPPORTED_EXPRESSION, Optional.of(query), format("Expected Pinot column handle %s to occur only once, but we have: %s", columnName, Joiner.on(",").join(handles))); + } + } + + ImmutableList.Builder outputIndices = ImmutableList.builder(); + for (Map.Entry expression : expressionsInPinotOrder.entrySet()) { + Integer index = nameToIndex.get(expression.getKey()); + if (hiddenColumnSet.contains(expression.getKey())) { + index = -1; // negative output index means to skip this value returned by pinot at query time + } + if (index == null) { + throw new PinotException( + PINOT_UNSUPPORTED_EXPRESSION, Optional.of(query), + format( + "Expected to find a Pinot column handle for the expression %s, but we have %s", + expression, + Joiner.on(",").withKeyValueSeparator(":").join(nameToIndex))); + } + outputIndices.add(index); + } + return outputIndices.build(); + } + + public LinkedHashMap getAssignments() + { + LinkedHashMap result = new LinkedHashMap<>(); + selections.entrySet().stream().filter(e -> !hiddenColumnSet.contains(e.getKey())).forEach(entry -> { + VariableReferenceExpression variable = entry.getKey(); + Selection selection = entry.getValue(); + PinotColumnHandle handle = selection.getOrigin() == Origin.TABLE_COLUMN ? new PinotColumnHandle(selection.getDefinition(), variable.getType(), PinotColumnHandle.PinotColumnType.REGULAR) : new PinotColumnHandle(variable, PinotColumnHandle.PinotColumnType.DERIVED); + result.put(variable, handle); + }); + return result; + } + + public PinotQueryGeneratorContext withOutputColumns(List outputColumns) + { + LinkedHashMap newSelections = new LinkedHashMap<>(); + outputColumns.forEach(o -> newSelections.put(o, requireNonNull(selections.get(o), String.format("Cannot find the selection %s in the original context %s", o, this)))); + + // Hidden columns flow as is from the previous + selections.entrySet().stream().filter(e -> hiddenColumnSet.contains(e.getKey())).forEach(e -> newSelections.put(e.getKey(), e.getValue())); + return new PinotQueryGeneratorContext(newSelections, from, filter, aggregations, groupByColumns, topNColumnOrderingMap, limit, variablesInAggregation, hiddenColumnSet); + } + + public PinotQueryGeneratorContext withVariablesInAggregation(Set newVariablesInAggregation) + { + return new PinotQueryGeneratorContext( + selections, + from, + filter, + aggregations, + groupByColumns, + topNColumnOrderingMap, + limit, + newVariablesInAggregation, + hiddenColumnSet); + } + + /** + * Where is the selection/projection originated from + */ + public enum Origin + { + TABLE_COLUMN, // refers to direct column in table + DERIVED, // expression is derived from one or more input columns or a combination of input columns and literals + LITERAL, // derived from literal + } + + // Projected/selected column definition in query + public static class Selection + { + private final String definition; + private final Origin origin; + + public Selection(String definition, Origin origin) + { + this.definition = definition; + this.origin = origin; + } + + public String getDefinition() + { + return definition; + } + + public Origin getOrigin() + { + return origin; + } + + @Override + public String toString() + { + return definition; + } + } +} diff --git a/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/MetadataUtil.java b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/MetadataUtil.java new file mode 100644 index 0000000000000..0a4eebde65806 --- /dev/null +++ b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/MetadataUtil.java @@ -0,0 +1,87 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.pinot; + +import com.facebook.airlift.json.JsonCodec; +import com.facebook.airlift.json.JsonCodecFactory; +import com.facebook.airlift.json.ObjectMapperProvider; +import com.facebook.presto.spi.type.StandardTypes; +import com.facebook.presto.spi.type.Type; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonDeserializer; +import com.fasterxml.jackson.databind.deser.std.FromStringDeserializer; +import com.google.common.collect.ImmutableMap; + +import java.util.List; +import java.util.Map; + +import static com.facebook.airlift.json.JsonCodec.listJsonCodec; +import static com.facebook.presto.spi.type.BigintType.BIGINT; +import static com.facebook.presto.spi.type.BooleanType.BOOLEAN; +import static com.facebook.presto.spi.type.DoubleType.DOUBLE; +import static com.facebook.presto.spi.type.IntegerType.INTEGER; +import static com.facebook.presto.spi.type.VarcharType.VARCHAR; +import static java.util.Locale.ENGLISH; +import static java.util.Objects.requireNonNull; + +public final class MetadataUtil +{ + public static final JsonCodec>> CATALOG_CODEC; + public static final JsonCodec TABLE_CODEC; + public static final JsonCodec COLUMN_CODEC; + public static final JsonCodec TABLES_JSON_CODEC; + public static final JsonCodec BROKERS_FOR_TABLE_JSON_CODEC; + public static final JsonCodec ROUTING_TABLES_JSON_CODEC; + public static final JsonCodec TIME_BOUNDARY_JSON_CODEC; + + private MetadataUtil() + { + } + + public static final class TestingTypeDeserializer + extends FromStringDeserializer + { + private final Map types = ImmutableMap.of( + StandardTypes.BOOLEAN, BOOLEAN, + StandardTypes.BIGINT, BIGINT, + StandardTypes.INTEGER, INTEGER, + StandardTypes.DOUBLE, DOUBLE, + StandardTypes.VARCHAR, VARCHAR); + + public TestingTypeDeserializer() + { + super(Type.class); + } + + @Override + protected Type _deserialize(String value, DeserializationContext context) + { + Type type = types.get(value.toLowerCase(ENGLISH)); + return requireNonNull(type, "Unknown type " + value); + } + } + + static { + ObjectMapperProvider objectMapperProvider = new ObjectMapperProvider(); + objectMapperProvider.setJsonDeserializers(ImmutableMap., JsonDeserializer>of(Type.class, new TestingTypeDeserializer())); + JsonCodecFactory codecFactory = new JsonCodecFactory(objectMapperProvider); + CATALOG_CODEC = codecFactory.mapJsonCodec(String.class, listJsonCodec(PinotTable.class)); + TABLE_CODEC = codecFactory.jsonCodec(PinotTable.class); + COLUMN_CODEC = codecFactory.jsonCodec(PinotColumnHandle.class); + TABLES_JSON_CODEC = codecFactory.jsonCodec(PinotClusterInfoFetcher.GetTables.class); + BROKERS_FOR_TABLE_JSON_CODEC = codecFactory.jsonCodec(PinotClusterInfoFetcher.BrokersForTable.class); + ROUTING_TABLES_JSON_CODEC = codecFactory.jsonCodec(PinotClusterInfoFetcher.RoutingTables.class); + TIME_BOUNDARY_JSON_CODEC = codecFactory.jsonCodec(PinotClusterInfoFetcher.TimeBoundary.class); + } +} diff --git a/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/MockPinotClusterInfoFetcher.java b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/MockPinotClusterInfoFetcher.java new file mode 100644 index 0000000000000..8d77f917d1e45 --- /dev/null +++ b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/MockPinotClusterInfoFetcher.java @@ -0,0 +1,420 @@ +/* + * 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.pinot; + +import com.facebook.airlift.http.client.testing.TestingHttpClient; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.pinot.common.data.Schema; + +import java.util.List; +import java.util.Map; + +import static com.facebook.presto.pinot.MetadataUtil.BROKERS_FOR_TABLE_JSON_CODEC; +import static com.facebook.presto.pinot.MetadataUtil.ROUTING_TABLES_JSON_CODEC; +import static com.facebook.presto.pinot.MetadataUtil.TABLES_JSON_CODEC; +import static com.facebook.presto.pinot.MetadataUtil.TIME_BOUNDARY_JSON_CODEC; + +public class MockPinotClusterInfoFetcher + extends PinotClusterInfoFetcher +{ + public MockPinotClusterInfoFetcher(PinotConfig pinotConfig) + { + super( + pinotConfig, + new PinotMetrics(), + new TestingHttpClient(request -> null), + TABLES_JSON_CODEC, + BROKERS_FOR_TABLE_JSON_CODEC, + ROUTING_TABLES_JSON_CODEC, + TIME_BOUNDARY_JSON_CODEC); + } + + @Override + public List getAllTables() + { + return ImmutableList.of(TestPinotSplitManager.realtimeOnlyTable.getTableName(), TestPinotSplitManager.hybridTable.getTableName()); + } + + @Override + public Map>> getRoutingTableForTable(String tableName) + { + ImmutableMap.Builder>> routingTable = ImmutableMap.builder(); + + if (TestPinotSplitManager.realtimeOnlyTable.getTableName().equalsIgnoreCase(tableName) || TestPinotSplitManager.hybridTable.getTableName().equalsIgnoreCase(tableName)) { + routingTable.put(tableName + "_REALTIME", ImmutableMap.of( + "server1", ImmutableList.of("segment11", "segment12"), + "server2", ImmutableList.of("segment21", "segment22"))); + } + + if (TestPinotSplitManager.hybridTable.getTableName().equalsIgnoreCase(tableName)) { + routingTable.put(tableName + "_OFFLINE", ImmutableMap.of( + "server3", ImmutableList.of("segment31", "segment32"), + "server4", ImmutableList.of("segment41", "segment42"))); + } + + return routingTable.build(); + } + + @Override + public Schema getTableSchema(String table) + throws Exception + { + // From the test pinot table airlineStats + return Schema.fromString("{\n" + + " \"schemaName\": \"airlineStats\",\n" + + " \"dimensionFieldSpecs\": [\n" + + " {\n" + + " \"name\": \"ActualElapsedTime\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"AirTime\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"AirlineID\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"ArrDel15\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"ArrDelay\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"ArrDelayMinutes\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"ArrTime\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"ArrTimeBlk\",\n" + + " \"dataType\": \"STRING\"\n" + + " },\n" + + " {\n" + + " \"name\": \"ArrivalDelayGroups\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"CRSArrTime\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"CRSDepTime\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"CRSElapsedTime\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"CancellationCode\",\n" + + " \"dataType\": \"STRING\"\n" + + " },\n" + + " {\n" + + " \"name\": \"Cancelled\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"Carrier\",\n" + + " \"dataType\": \"STRING\"\n" + + " },\n" + + " {\n" + + " \"name\": \"CarrierDelay\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"DayOfWeek\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"DayofMonth\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"DepDel15\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"DepDelay\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"DepDelayMinutes\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"DepTime\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"DepTimeBlk\",\n" + + " \"dataType\": \"STRING\"\n" + + " },\n" + + " {\n" + + " \"name\": \"DepartureDelayGroups\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"Dest\",\n" + + " \"dataType\": \"STRING\"\n" + + " },\n" + + " {\n" + + " \"name\": \"DestAirportID\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"DestAirportSeqID\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"DestCityMarketID\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"DestCityName\",\n" + + " \"dataType\": \"STRING\"\n" + + " },\n" + + " {\n" + + " \"name\": \"DestState\",\n" + + " \"dataType\": \"STRING\"\n" + + " },\n" + + " {\n" + + " \"name\": \"DestStateFips\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"DestStateName\",\n" + + " \"dataType\": \"STRING\"\n" + + " },\n" + + " {\n" + + " \"name\": \"DestWac\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"Distance\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"DistanceGroup\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"DivActualElapsedTime\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"DivAirportIDs\",\n" + + " \"dataType\": \"INT\",\n" + + " \"singleValueField\": false\n" + + " },\n" + + " {\n" + + " \"name\": \"DivAirportLandings\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"DivAirportSeqIDs\",\n" + + " \"dataType\": \"INT\",\n" + + " \"singleValueField\": false\n" + + " },\n" + + " {\n" + + " \"name\": \"DivAirports\",\n" + + " \"dataType\": \"STRING\",\n" + + " \"singleValueField\": false\n" + + " },\n" + + " {\n" + + " \"name\": \"DivArrDelay\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"DivDistance\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"DivLongestGTimes\",\n" + + " \"dataType\": \"INT\",\n" + + " \"singleValueField\": false\n" + + " },\n" + + " {\n" + + " \"name\": \"DivReachedDest\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"DivTailNums\",\n" + + " \"dataType\": \"STRING\",\n" + + " \"singleValueField\": false\n" + + " },\n" + + " {\n" + + " \"name\": \"DivTotalGTimes\",\n" + + " \"dataType\": \"INT\",\n" + + " \"singleValueField\": false\n" + + " },\n" + + " {\n" + + " \"name\": \"DivWheelsOffs\",\n" + + " \"dataType\": \"INT\",\n" + + " \"singleValueField\": false\n" + + " },\n" + + " {\n" + + " \"name\": \"DivWheelsOns\",\n" + + " \"dataType\": \"INT\",\n" + + " \"singleValueField\": false\n" + + " },\n" + + " {\n" + + " \"name\": \"Diverted\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"FirstDepTime\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"FlightDate\",\n" + + " \"dataType\": \"STRING\"\n" + + " },\n" + + " {\n" + + " \"name\": \"FlightNum\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"Flights\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"LateAircraftDelay\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"LongestAddGTime\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"Month\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"NASDelay\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"Origin\",\n" + + " \"dataType\": \"STRING\"\n" + + " },\n" + + " {\n" + + " \"name\": \"OriginAirportID\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"OriginAirportSeqID\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"OriginCityMarketID\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"OriginCityName\",\n" + + " \"dataType\": \"STRING\"\n" + + " },\n" + + " {\n" + + " \"name\": \"OriginState\",\n" + + " \"dataType\": \"STRING\"\n" + + " },\n" + + " {\n" + + " \"name\": \"OriginStateFips\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"OriginStateName\",\n" + + " \"dataType\": \"STRING\"\n" + + " },\n" + + " {\n" + + " \"name\": \"OriginWac\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"Quarter\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"RandomAirports\",\n" + + " \"dataType\": \"STRING\",\n" + + " \"singleValueField\": false\n" + + " },\n" + + " {\n" + + " \"name\": \"SecurityDelay\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"TailNum\",\n" + + " \"dataType\": \"STRING\"\n" + + " },\n" + + " {\n" + + " \"name\": \"TaxiIn\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"TaxiOut\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"Year\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"WheelsOn\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"WheelsOff\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"WeatherDelay\",\n" + + " \"dataType\": \"INT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"UniqueCarrier\",\n" + + " \"dataType\": \"STRING\"\n" + + " },\n" + + " {\n" + + " \"name\": \"TotalAddGTime\",\n" + + " \"dataType\": \"INT\"\n" + + " }\n" + + " ],\n" + + " \"timeFieldSpec\": {\n" + + " \"incomingGranularitySpec\": {\n" + + " \"name\": \"DaysSinceEpoch\",\n" + + " \"dataType\": \"INT\",\n" + + " \"timeType\": \"DAYS\"\n" + + " }\n" + + " },\n" + + " \"updateSemantic\": null\n" + + "}"); + } + + @Override + public TimeBoundary getTimeBoundaryForTable(String table) + { + if (TestPinotSplitManager.hybridTable.getTableName().equalsIgnoreCase(table)) { + return new TimeBoundary("secondsSinceEpoch", "4562345"); + } + + return new TimeBoundary(); + } +} diff --git a/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotBrokerPageSource.java b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotBrokerPageSource.java new file mode 100644 index 0000000000000..56034a7ad9f71 --- /dev/null +++ b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotBrokerPageSource.java @@ -0,0 +1,205 @@ +/* + * 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.pinot; + +import com.facebook.airlift.json.ObjectMapperProvider; +import com.facebook.presto.pinot.query.PinotQueryGenerator; +import com.facebook.presto.spi.PageBuilder; +import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.block.BlockBuilder; +import com.facebook.presto.spi.type.Type; +import com.facebook.presto.testing.TestingConnectorSession; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Streams; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +import java.io.IOException; +import java.util.HashSet; +import java.util.List; +import java.util.Optional; +import java.util.Set; + +import static com.facebook.presto.spi.type.BigintType.BIGINT; +import static com.facebook.presto.spi.type.VarcharType.VARCHAR; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; + +public class TestPinotBrokerPageSource + extends TestPinotQueryBase +{ + private static PinotTableHandle pinotTable = new PinotTableHandle("connId", "schema", "tbl"); + private final ObjectMapper objectMapper = new ObjectMapperProvider().get(); + + private static class PqlParsedInfo + { + final int groupByColumns; + final int columns; + final int rows; + + private PqlParsedInfo(int groupByColumns, int columns, int rows) + { + this.groupByColumns = groupByColumns; + this.columns = columns; + this.rows = rows; + } + + public static PqlParsedInfo forSelection(int columns, int rows) + { + return new PqlParsedInfo(0, columns, rows); + } + + public static PqlParsedInfo forAggregation(int groups, int aggregates, int rows) + { + return new PqlParsedInfo(groups, groups + aggregates, rows); + } + } + + PqlParsedInfo getBasicInfoFromPql(String pqlResponse) + throws IOException + { + JsonNode pqlJson = objectMapper.readTree(pqlResponse); + JsonNode selectionResults = pqlJson.get("selectionResults"); + if (selectionResults != null) { + return PqlParsedInfo.forSelection(selectionResults.get("columns").size(), selectionResults.get("results").size()); + } + + JsonNode aggregationResults = pqlJson.get("aggregationResults"); + int aggregates = aggregationResults.size(); + Set> groups = new HashSet<>(); + int groupByColumns = 0; + int pureAggregates = 0; + for (int i = 0; i < aggregates; i++) { + JsonNode groupByResult = aggregationResults.get(i).get("groupByResult"); + if (groupByResult != null) { + for (int j = 0; j < groupByResult.size(); ++j) { + JsonNode groupJson = groupByResult.get(j).get("group"); + List group = Streams.stream(groupJson.iterator()).map(JsonNode::asText).collect(toImmutableList()); + groups.add(group); + if (groupByColumns == 0) { + groupByColumns = group.size(); + } + } + } + else { + pureAggregates++; + } + } + assertTrue(pureAggregates == 0 || pureAggregates == aggregates, String.format("In pql response %s, got mixed aggregates %d of %d", pqlResponse, pureAggregates, aggregates)); + if (pureAggregates == 0) { + return PqlParsedInfo.forAggregation(groupByColumns, aggregates, groups.size()); + } + return PqlParsedInfo.forAggregation(0, pureAggregates, 1); + } + + @DataProvider(name = "pqlResponses") + public static Object[][] pqlResponsesProvider() + { + return new Object[][] { + {"SELECT count(*), sum(regionId) FROM eats_job_state GROUP BY jobState TOP 1000000", + "{\"aggregationResults\":[{\"groupByResult\":[{\"value\":\"10646777\",\"group\":[\"CREATED\"]},{\"value\":\"9441201\",\"group\":[\"ASSIGNED\"]},{\"value\":\"5329962\",\"group\":[\"SUBMITTED_TO_BILLING\"]},{\"value\":\"5281666\",\"group\":[\"PICKUP_COMPLETED\"]},{\"value\":\"5225839\",\"group\":[\"OFFERED\"]},{\"value\":\"5088568\",\"group\":[\"READY\"]},{\"value\":\"5027369\",\"group\":[\"COMPLETED\"]},{\"value\":\"3677267\",\"group\":[\"SUBMITTED_TO_MANIFEST\"]},{\"value\":\"1559953\",\"group\":[\"SCHEDULED\"]},{\"value\":\"1532913\",\"group\":[\"ACCEPTED\"]},{\"value\":\"1532891\",\"group\":[\"RELEASED\"]},{\"value\":\"531719\",\"group\":[\"UNASSIGNED\"]},{\"value\":\"252977\",\"group\":[\"PREP_TIME_UPDATED\"]},{\"value\":\"243463\",\"group\":[\"CANCELED\"]},{\"value\":\"211553\",\"group\":[\"PAYMENT_PENDING\"]},{\"value\":\"148548\",\"group\":[\"PAYMENT_CONFIRMED\"]},{\"value\":\"108057\",\"group\":[\"UNFULFILLED_WARNED\"]},{\"value\":\"47043\",\"group\":[\"DELIVERY_FAILED\"]},{\"value\":\"30832\",\"group\":[\"UNFULFILLED\"]},{\"value\":\"18009\",\"group\":[\"SCHEDULE_ORDER_CREATED\"]},{\"value\":\"16459\",\"group\":[\"SCHEDULE_ORDER_ACCEPTED\"]},{\"value\":\"11086\",\"group\":[\"FAILED\"]},{\"value\":\"9976\",\"group\":[\"SCHEDULE_ORDER_OFFERED\"]},{\"value\":\"3094\",\"group\":[\"PAYMENT_FAILED\"]}],\"function\":\"count_star\",\"groupByColumns\":[\"jobState\"]},{\"groupByResult\":[{\"value\":\"3274799599.00000\",\"group\":[\"CREATED\"]},{\"value\":\"2926585674.00000\",\"group\":[\"ASSIGNED\"]},{\"value\":\"1645707788.00000\",\"group\":[\"SUBMITTED_TO_BILLING\"]},{\"value\":\"1614715326.00000\",\"group\":[\"OFFERED\"]},{\"value\":\"1608041994.00000\",\"group\":[\"PICKUP_COMPLETED\"]},{\"value\":\"1568036720.00000\",\"group\":[\"READY\"]},{\"value\":\"1541977381.00000\",\"group\":[\"COMPLETED\"]},{\"value\":\"1190457213.00000\",\"group\":[\"SUBMITTED_TO_MANIFEST\"]},{\"value\":\"430246171.00000\",\"group\":[\"SCHEDULED\"]},{\"value\":\"422020881.00000\",\"group\":[\"RELEASED\"]},{\"value\":\"421937782.00000\",\"group\":[\"ACCEPTED\"]},{\"value\":\"147557783.00000\",\"group\":[\"UNASSIGNED\"]},{\"value\":\"94882088.00000\",\"group\":[\"PREP_TIME_UPDATED\"]},{\"value\":\"86447788.00000\",\"group\":[\"CANCELED\"]},{\"value\":\"77505566.00000\",\"group\":[\"PAYMENT_PENDING\"]},{\"value\":\"53955037.00000\",\"group\":[\"PAYMENT_CONFIRMED\"]},{\"value\":\"36026660.00000\",\"group\":[\"UNFULFILLED_WARNED\"]},{\"value\":\"15306755.00000\",\"group\":[\"DELIVERY_FAILED\"]},{\"value\":\"8811788.00000\",\"group\":[\"UNFULFILLED\"]},{\"value\":\"5301567.00000\",\"group\":[\"SCHEDULE_ORDER_CREATED\"]},{\"value\":\"4855342.00000\",\"group\":[\"SCHEDULE_ORDER_ACCEPTED\"]},{\"value\":\"3113490.00000\",\"group\":[\"FAILED\"]},{\"value\":\"2811789.00000\",\"group\":[\"SCHEDULE_ORDER_OFFERED\"]},{\"value\":\"1053944.00000\",\"group\":[\"PAYMENT_FAILED\"]}],\"function\":\"sum_regionId\",\"groupByColumns\":[\"jobState\"]}],\"exceptions\":[],\"numServersQueried\":7,\"numServersResponded\":7,\"numDocsScanned\":55977222,\"numEntriesScannedInFilter\":0,\"numEntriesScannedPostFilter\":111954444,\"totalDocs\":55977222,\"numGroupsLimitReached\":false,\"timeUsedMs\":775,\"segmentStatistics\":[],\"traceInfo\":{}}", + ImmutableList.of(VARCHAR, BIGINT, BIGINT), Optional.empty()}, + {"SELECT count(*) FROM eats_job_state GROUP BY jobState TOP 1000000", + "{\"traceInfo\":{},\"numEntriesScannedPostFilter\":55979949,\"numDocsScanned\":55979949,\"numServersResponded\":7,\"numGroupsLimitReached\":false,\"aggregationResults\":[{\"groupByResult\":[{\"value\":\"10647363\",\"group\":[\"CREATED\"]},{\"value\":\"9441638\",\"group\":[\"ASSIGNED\"]},{\"value\":\"5330203\",\"group\":[\"SUBMITTED_TO_BILLING\"]},{\"value\":\"5281905\",\"group\":[\"PICKUP_COMPLETED\"]},{\"value\":\"5226090\",\"group\":[\"OFFERED\"]},{\"value\":\"5088813\",\"group\":[\"READY\"]},{\"value\":\"5027589\",\"group\":[\"COMPLETED\"]},{\"value\":\"3677424\",\"group\":[\"SUBMITTED_TO_MANIFEST\"]},{\"value\":\"1560029\",\"group\":[\"SCHEDULED\"]},{\"value\":\"1533006\",\"group\":[\"ACCEPTED\"]},{\"value\":\"1532980\",\"group\":[\"RELEASED\"]},{\"value\":\"531745\",\"group\":[\"UNASSIGNED\"]},{\"value\":\"252989\",\"group\":[\"PREP_TIME_UPDATED\"]},{\"value\":\"243477\",\"group\":[\"CANCELED\"]},{\"value\":\"211571\",\"group\":[\"PAYMENT_PENDING\"]},{\"value\":\"148557\",\"group\":[\"PAYMENT_CONFIRMED\"]},{\"value\":\"108062\",\"group\":[\"UNFULFILLED_WARNED\"]},{\"value\":\"47048\",\"group\":[\"DELIVERY_FAILED\"]},{\"value\":\"30832\",\"group\":[\"UNFULFILLED\"]},{\"value\":\"18009\",\"group\":[\"SCHEDULE_ORDER_CREATED\"]},{\"value\":\"16461\",\"group\":[\"SCHEDULE_ORDER_ACCEPTED\"]},{\"value\":\"11086\",\"group\":[\"FAILED\"]},{\"value\":\"9978\",\"group\":[\"SCHEDULE_ORDER_OFFERED\"]},{\"value\":\"3094\",\"group\":[\"PAYMENT_FAILED\"]}],\"function\":\"count_star\",\"groupByColumns\":[\"jobState\"]}],\"exceptions\":[],\"numEntriesScannedInFilter\":0,\"timeUsedMs\":402,\"segmentStatistics\":[],\"numServersQueried\":7,\"totalDocs\":55979949}", + ImmutableList.of(VARCHAR, BIGINT), Optional.empty()}, + {"SELECT count(*) FROM eats_job_state", + "{\"traceInfo\":{},\"numEntriesScannedPostFilter\":0,\"numDocsScanned\":55981101,\"numServersResponded\":7,\"numGroupsLimitReached\":false,\"aggregationResults\":[{\"function\":\"count_star\",\"value\":\"55981101\"}],\"exceptions\":[],\"numEntriesScannedInFilter\":0,\"timeUsedMs\":7,\"segmentStatistics\":[],\"numServersQueried\":7,\"totalDocs\":55981101}", + ImmutableList.of(BIGINT), Optional.empty()}, + {"SELECT sum(regionId), count(*) FROM eats_job_state", + "{\"traceInfo\":{},\"numEntriesScannedPostFilter\":55981641,\"numDocsScanned\":55981641,\"numServersResponded\":7,\"numGroupsLimitReached\":false,\"aggregationResults\":[{\"function\":\"sum_regionId\",\"value\":\"17183585871.00000\"},{\"function\":\"count_star\",\"value\":\"55981641\"}],\"exceptions\":[],\"numEntriesScannedInFilter\":0,\"timeUsedMs\":549,\"segmentStatistics\":[],\"numServersQueried\":7,\"totalDocs\":55981641}", + ImmutableList.of(BIGINT, BIGINT), Optional.empty()}, + {"SELECT jobState, regionId FROM eats_job_state LIMIT 10", + "{\"selectionResults\":{\"columns\":[\"jobState\",\"regionId\"],\"results\":[[\"CREATED\",\"197\"],[\"SUBMITTED_TO_BILLING\",\"227\"],[\"ASSIGNED\",\"188\"],[\"SCHEDULED\",\"1479\"],[\"CANCELED\",\"1708\"],[\"CREATED\",\"134\"],[\"CREATED\",\"12\"],[\"OFFERED\",\"30\"],[\"COMPLETED\",\"215\"],[\"CREATED\",\"7\"]]},\"exceptions\":[],\"numServersQueried\":7,\"numServersResponded\":7,\"numDocsScanned\":380,\"numEntriesScannedInFilter\":0,\"numEntriesScannedPostFilter\":760,\"totalDocs\":55988817,\"numGroupsLimitReached\":false,\"timeUsedMs\":2,\"segmentStatistics\":[],\"traceInfo\":{}}", + ImmutableList.of(VARCHAR, BIGINT), Optional.empty()}, + {"SELECT shoppingCartUUID, $validUntil, $validFrom, jobState, tenancy, accountUUID, vehicleViewId, $partition, clientUUID, orderJobUUID, productTypeUUID, demandJobUUID, regionId, workflowUUID, jobType, kafkaOffset, productUUID, timestamp, flowType, ts FROM eats_job_state LIMIT 10", + "{\"selectionResults\":{\"columns\":[\"shoppingCartUUID\",\"$validUntil\",\"$validFrom\",\"jobState\",\"tenancy\",\"accountUUID\",\"vehicleViewId\",\"$partition\",\"clientUUID\",\"orderJobUUID\",\"productTypeUUID\",\"demandJobUUID\",\"regionId\",\"workflowUUID\",\"jobType\",\"kafkaOffset\",\"productUUID\",\"timestamp\",\"flowType\",\"ts\"],\"results\":[]},\"traceInfo\":{},\"numEntriesScannedPostFilter\":0,\"numDocsScanned\":0,\"numServersResponded\":7,\"numGroupsLimitReached\":false,\"exceptions\":[{\"errorCode\":200,\"message\":\"QueryExecutionError:\\njava.lang.NullPointerException\\n\\tat java.lang.Class.forName0(Native Method\\n\\tat\"}],\"numEntriesScannedInFilter\":0,\"timeUsedMs\":3,\"segmentStatistics\":[],\"numServersQueried\":7,\"totalDocs\":0}", + ImmutableList.of(), Optional.of(PinotException.class)}, + {"SELECT * from eats_utilization_summarized", + "{\n" + + " \"selectionResults\": {\n" + + " \"columns\": [\"activeTrips\", \"numDrivers\", \"region\", \"rowtime\", \"secondsSinceEpoch\", \"utilization\", \"utilizedDrivers\", \"vehicleViewId\", \"windowEnd\", \"windowStart\"],\n" + + " \"results\": [\n" + + " [\"0\", \"0\", \"foobar\", \"null\", \"4588780800\", \"-∞\", \"0\", \"20017545\", \"4588780740000\", \"4588780725000\"],\n" + + " [\"8699\", \"11452\", \"doobar\", \"null\", \"4588780800\", \"0.730701685\", \"8368\", \"0\", \"4588780740000\", \"4588780725000\"],\n" + + " [\"0\", \"14\", \"zoobar\", \"null\", \"4588780800\", \"0.5\", \"7\", \"20014789\", \"4588780740000\", \"4588780725000\"],\n" + + " [\"0\", \"23\", \"moobar\", \"null\", \"4588780800\", \"0.4336180091\", \"10\", \"20009983\", \"4588780740000\", \"4588780725000\"],\n" + + " [\"0\", \"840\", \"koobar\", \"null\", \"4588780800\", \"0.6597985029\", \"554\", \"20006875\", \"4588780740000\", \"4588780725000\"],\n" + + " [\"0\", \"0\", \"loobar\", \"null\", \"4588780800\", \"-∞\", \"0\", \"20006291\", \"4588780740000\", \"4588780725000\"],\n" + + " [\"15\", \"1832\", \"monkeybar\", \"null\", \"4588780800\", \"0.8792306185\", \"1610\", \"20004007\", \"4588780740000\", \"4588780725000\"],\n" + + " [\"0\", \"0\", \"donkeybar\", \"null\", \"4588780800\", \"-∞\", \"0\", \"0\", \"4588780740000\", \"4588780725000\"],\n" + + " [\"1\", \"7\", \"horseybar\", \"null\", \"4588780800\", \"0.2857142985\", \"2\", \"20016753\", \"4588780740000\", \"4588780725000\"],\n" + + " [\"0\", \"130\", \"ginbar\", \"null\", \"4588780800\", \"0.8052611947\", \"105\", \"10000942\", \"4588780740000\", \"4588780725000\"]\n" + + " ]\n" + + " },\n" + + " \"exceptions\": [],\n" + + " \"numServersQueried\": 4,\n" + + " \"numServersResponded\": 4,\n" + + " \"numSegmentsQueried\": 24,\n" + + " \"numSegmentsProcessed\": 24,\n" + + " \"numSegmentsMatched\": 24,\n" + + " \"numDocsScanned\": 240,\n" + + " \"numEntriesScannedInFilter\": 0,\n" + + " \"numEntriesScannedPostFilter\": 240,\n" + + " \"numGroupsLimitReached\": false,\n" + + " \"totalDocs\": 1000,\n" + + " \"timeUsedMs\": 6,\n" + + " \"segmentStatistics\": [],\n" + + " \"traceInfo\": {}\n" + + "}", + ImmutableList.of(BIGINT, BIGINT, VARCHAR, VARCHAR, BIGINT, BIGINT, BIGINT, BIGINT, BIGINT, BIGINT), Optional.empty()} + }; + } + + @Test(dataProvider = "pqlResponses") + public void testPopulateFromPql(String pql, String pqlResponse, List types, Optional> expectedError) + throws IOException + { + PqlParsedInfo pqlParsedInfo = getBasicInfoFromPql(pqlResponse); + ImmutableList.Builder blockBuilders = ImmutableList.builder(); + PageBuilder pageBuilder = new PageBuilder(types); + PinotBrokerPageSource pageSource = getPinotBrokerPageSource(); + for (int i = 0; i < types.size(); i++) { + blockBuilders.add(pageBuilder.getBlockBuilder(i)); + } + + Optional thrown = Optional.empty(); + int rows = -1; + try { + rows = pageSource.populateFromPqlResults(pql, pqlParsedInfo.groupByColumns, blockBuilders.build(), types, pqlResponse); + } + catch (PrestoException e) { + thrown = Optional.of(e); + } + + Optional> thrownType = thrown.map(e -> e.getClass()); + Optional errorString = thrown.map(e -> Throwables.getStackTraceAsString(e)); + assertEquals(thrownType, expectedError, String.format("Expected error %s, but got error of type %s: %s", expectedError, thrownType, errorString)); + if (!expectedError.isPresent()) { + assertEquals(types.size(), pqlParsedInfo.columns); + assertEquals(rows, pqlParsedInfo.rows); + } + } + + private PinotBrokerPageSource getPinotBrokerPageSource() + { + List pinotColumnHandles = ImmutableList.of(regionId, fare, city, fare, secondsSinceEpoch); + PinotConfig pinotConfig = new PinotConfig(); + PinotQueryGenerator.GeneratedPql generatedPql = new PinotQueryGenerator.GeneratedPql(pinotTable.getTableName(), String.format("SELECT %s, %s FROM %s LIMIT %d", city.getColumnName(), regionId.getColumnName(), pinotTable.getTableName(), pinotConfig.getLimitLargeForSegment()), ImmutableList.of(0, 1), 0, false, true); + return new PinotBrokerPageSource(pinotConfig, new TestingConnectorSession(ImmutableList.of()), generatedPql, pinotColumnHandles, new MockPinotClusterInfoFetcher(pinotConfig), objectMapper); + } +} diff --git a/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotClusterInfoFetcher.java b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotClusterInfoFetcher.java new file mode 100644 index 0000000000000..e4493b3831c63 --- /dev/null +++ b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotClusterInfoFetcher.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.pinot; + +import com.facebook.airlift.http.client.HttpClient; +import com.facebook.airlift.http.client.HttpStatus; +import com.facebook.airlift.http.client.testing.TestingHttpClient; +import com.facebook.airlift.http.client.testing.TestingResponse; +import com.facebook.presto.testing.assertions.Assert; +import com.google.common.collect.ImmutableSet; +import com.google.common.net.MediaType; +import io.airlift.units.Duration; +import org.testng.annotations.Test; + +import java.util.concurrent.TimeUnit; + +public class TestPinotClusterInfoFetcher +{ + @Test + public void testBrokersParsed() + { + HttpClient httpClient = new TestingHttpClient((request) -> TestingResponse.mockResponse(HttpStatus.OK, MediaType.JSON_UTF_8, "{\n" + + " \"tableName\": \"dummy\",\n" + + " \"brokers\": [\n" + + " {\n" + + " \"tableType\": \"offline\",\n" + + " \"instances\": [\n" + + " \"Broker_dummy-broker-host1-datacenter1_6513\",\n" + + " \"Broker_dummy-broker-host2-datacenter1_6513\",\n" + + " \"Broker_dummy-broker-host4-datacenter1_6513\"\n" + + " ]\n" + + " },\n" + + " {\n" + + " \"tableType\": \"realtime\",\n" + + " \"instances\": [\n" + + " \"Broker_dummy-broker-host1-datacenter1_6513\",\n" + + " \"Broker_dummy-broker-host2-datacenter1_6513\",\n" + + " \"Broker_dummy-broker-host3-datacenter1_6513\"\n" + + " ]\n" + + " }\n" + + " ],\n" + + " \"server\": [\n" + + " {\n" + + " \"tableType\": \"offline\",\n" + + " \"instances\": [\n" + + " \"Server_dummy-server-host8-datacenter1_7090\",\n" + + " \"Server_dummy-server-host9-datacenter1_7090\"\n" + + " ]\n" + + " },\n" + + " {\n" + + " \"tableType\": \"realtime\",\n" + + " \"instances\": [\n" + + " \"Server_dummy-server-host7-datacenter1_7090\",\n" + + " \"Server_dummy-server-host4-datacenter1_7090\",\n" + + " \"Server_dummy-server-host5-datacenter1_7090\",\n" + + " \"Server_dummy-server-host6-datacenter1_7090\"\n" + + " ]\n" + + " }\n" + + " ]\n" + + "}")); + PinotConfig pinotConfig = new PinotConfig() + .setMetadataCacheExpiry(new Duration(0, TimeUnit.MILLISECONDS)) + .setControllerUrls("localhost:7900"); + PinotClusterInfoFetcher pinotClusterInfoFetcher = new PinotClusterInfoFetcher(pinotConfig, new PinotMetrics(), httpClient, MetadataUtil.TABLES_JSON_CODEC, MetadataUtil.BROKERS_FOR_TABLE_JSON_CODEC, MetadataUtil.ROUTING_TABLES_JSON_CODEC, MetadataUtil.TIME_BOUNDARY_JSON_CODEC); + ImmutableSet brokers = ImmutableSet.copyOf(pinotClusterInfoFetcher.getAllBrokersForTable("dummy")); + Assert.assertEquals(ImmutableSet.of("dummy-broker-host1-datacenter1:6513", "dummy-broker-host2-datacenter1:6513", "dummy-broker-host3-datacenter1:6513", "dummy-broker-host4-datacenter1:6513"), brokers); + } +} diff --git a/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotColumnHandle.java b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotColumnHandle.java new file mode 100644 index 0000000000000..b6b50bc3f6e12 --- /dev/null +++ b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotColumnHandle.java @@ -0,0 +1,55 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.pinot; + +import com.facebook.airlift.testing.EquivalenceTester; +import org.testng.annotations.Test; + +import static com.facebook.presto.pinot.MetadataUtil.COLUMN_CODEC; +import static com.facebook.presto.pinot.PinotColumnHandle.PinotColumnType.DERIVED; +import static com.facebook.presto.pinot.PinotColumnHandle.PinotColumnType.REGULAR; +import static com.facebook.presto.spi.type.BigintType.BIGINT; +import static com.facebook.presto.spi.type.VarcharType.VARCHAR; +import static org.testng.Assert.assertEquals; + +public class TestPinotColumnHandle +{ + private final PinotColumnHandle columnHandle = new PinotColumnHandle("columnName", VARCHAR, REGULAR); + + @Test + public void testJsonRoundTrip() + { + String json = COLUMN_CODEC.toJson(columnHandle); + PinotColumnHandle copy = COLUMN_CODEC.fromJson(json); + assertEquals(copy, columnHandle); + } + + @Test + public void testEquivalence() + { + EquivalenceTester + .equivalenceTester() + .addEquivalentGroup( + new PinotColumnHandle("columnName", VARCHAR, REGULAR), + new PinotColumnHandle("columnName", VARCHAR, DERIVED), + new PinotColumnHandle("columnName", BIGINT, REGULAR), + new PinotColumnHandle("columnName", BIGINT, DERIVED)) + .addEquivalentGroup( + new PinotColumnHandle("columnNameX", VARCHAR, REGULAR), + new PinotColumnHandle("columnNameX", VARCHAR, DERIVED), + new PinotColumnHandle("columnNameX", BIGINT, REGULAR), + new PinotColumnHandle("columnNameX", BIGINT, DERIVED)) + .check(); + } +} diff --git a/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotConfig.java b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotConfig.java new file mode 100644 index 0000000000000..fe3bae05965c7 --- /dev/null +++ b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotConfig.java @@ -0,0 +1,123 @@ +/* + * 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.pinot; + +import com.facebook.airlift.configuration.testing.ConfigAssertions; +import com.google.common.collect.ImmutableMap; +import io.airlift.units.Duration; +import org.testng.annotations.Test; + +import java.util.Map; +import java.util.concurrent.TimeUnit; + +public class TestPinotConfig +{ + @Test + public void testDefaults() + { + ConfigAssertions.assertRecordedDefaults( + ConfigAssertions.recordDefaults(PinotConfig.class) + .setExtraHttpHeaders("") + .setControllerUrls("") + .setIdleTimeout(new Duration(5, TimeUnit.MINUTES)) + .setLimitLargeForSegment(PinotConfig.DEFAULT_LIMIT_LARGE_FOR_SEGMENT) + .setTopNLarge(PinotConfig.DEFAULT_TOPN_LARGE) + .setMaxBacklogPerServer(PinotConfig.DEFAULT_MAX_BACKLOG_PER_SERVER) + .setMaxConnectionsPerServer(PinotConfig.DEFAULT_MAX_CONNECTIONS_PER_SERVER) + .setMinConnectionsPerServer(PinotConfig.DEFAULT_MIN_CONNECTIONS_PER_SERVER) + .setThreadPoolSize(PinotConfig.DEFAULT_THREAD_POOL_SIZE) + .setEstimatedSizeInBytesForNonNumericColumn(20) + .setConnectionTimeout(new Duration(1, TimeUnit.MINUTES)) + .setControllerRestService(null) + .setServiceHeaderParam("RPC-Service") + .setCallerHeaderValue("presto") + .setCallerHeaderParam("RPC-Caller") + .setMetadataCacheExpiry(new Duration(2, TimeUnit.MINUTES)) + .setAllowMultipleAggregations(false) + .setPreferBrokerQueries(true) + .setRestProxyServiceForQuery(null) + .setRestProxyUrl(null) + .setNumSegmentsPerSplit(1) + .setFetchRetryCount(2) + .setIgnoreEmptyResponses(false) + .setUseDateTrunc(false) + .setForbidSegmentQueries(false) + .setNonAggregateLimitForBrokerQueries(PinotConfig.DEFAULT_NON_AGGREGATE_LIMIT_FOR_BROKER_QUERIES) + .setUseDateTrunc(false)); + } + + @Test + public void testExplicitPropertyMappings() + { + Map properties = new ImmutableMap.Builder() + .put("pinot.extra-http-headers", "k:v") + .put("pinot.controller-rest-service", "pinot-controller-service") + .put("pinot.controller-urls", "host1:1111,host2:1111") + .put("pinot.idle-timeout", "1h") + .put("pinot.topn-large", "1000") + .put("pinot.max-backlog-per-server", "15") + .put("pinot.max-connections-per-server", "10") + .put("pinot.min-connections-per-server", "1") + .put("pinot.thread-pool-size", "100") + .put("pinot.estimated-size-in-bytes-for-non-numeric-column", "30") + .put("pinot.connection-timeout", "8m") + .put("pinot.metadata-expiry", "1m") + .put("pinot.caller-header-value", "myCaller") + .put("pinot.caller-header-param", "myParam") + .put("pinot.service-header-param", "myServiceHeader") + .put("pinot.allow-multiple-aggregations", "true") + .put("pinot.prefer-broker-queries", "false") + .put("pinot.rest-proxy-url", "localhost:1111") + .put("pinot.rest-proxy-service-for-query", "pinot-rest-proxy-service") + .put("pinot.num-segments-per-split", "2") + .put("pinot.ignore-empty-responses", "true") + .put("pinot.fetch-retry-count", "3") + .put("pinot.non-aggregate-limit-for-broker-queries", "10") + .put("pinot.use-date-trunc", "true") + .put("pinot.limit-large-for-segment", "100") + .put("pinot.forbid-segment-queries", "true") + .build(); + + PinotConfig expected = new PinotConfig() + .setExtraHttpHeaders("k:v") + .setControllerRestService("pinot-controller-service") + .setControllerUrls("host1:1111,host2:1111") + .setRestProxyUrl("localhost:1111") + .setIdleTimeout(new Duration(1, TimeUnit.HOURS)) + .setLimitLargeForSegment(100000) + .setTopNLarge(1000) + .setMaxBacklogPerServer(15) + .setMaxConnectionsPerServer(10) + .setMinConnectionsPerServer(1) + .setThreadPoolSize(100) + .setEstimatedSizeInBytesForNonNumericColumn(30) + .setConnectionTimeout(new Duration(8, TimeUnit.MINUTES)) + .setServiceHeaderParam("myServiceHeader") + .setCallerHeaderValue("myCaller") + .setCallerHeaderParam("myParam") + .setMetadataCacheExpiry(new Duration(1, TimeUnit.MINUTES)) + .setAllowMultipleAggregations(true) + .setPreferBrokerQueries(false) + .setRestProxyServiceForQuery("pinot-rest-proxy-service") + .setNumSegmentsPerSplit(2) + .setIgnoreEmptyResponses(true) + .setFetchRetryCount(3) + .setNonAggregateLimitForBrokerQueries(10) + .setLimitLargeForSegment(100) + .setForbidSegmentQueries(true) + .setUseDateTrunc(true); + + ConfigAssertions.assertFullMapping(properties, expected); + } +} diff --git a/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotMetadata.java b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotMetadata.java new file mode 100644 index 0000000000000..9ec5185124f67 --- /dev/null +++ b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotMetadata.java @@ -0,0 +1,46 @@ +/* + * 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.pinot; + +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.SchemaTableName; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import org.testng.annotations.Test; + +import java.util.List; +import java.util.concurrent.Executors; + +import static org.testng.Assert.assertEquals; + +public class TestPinotMetadata +{ + private final PinotConfig pinotConfig = new PinotConfig(); + private final PinotConnection pinotConnection = new PinotConnection(new MockPinotClusterInfoFetcher(pinotConfig), pinotConfig, Executors.newSingleThreadExecutor()); + private final PinotMetadata metadata = new PinotMetadata(TestPinotSplitManager.pinotConnectorId, pinotConnection); + + @Test + public void testTables() + { + ConnectorSession session = TestPinotSplitManager.createSessionWithNumSplits(1, false, pinotConfig); + List schemaTableNames = metadata.listTables(session, (String) null); + assertEquals(ImmutableSet.copyOf(schemaTableNames), ImmutableSet.of(new SchemaTableName("default", TestPinotSplitManager.realtimeOnlyTable.getTableName()), new SchemaTableName("default", TestPinotSplitManager.hybridTable.getTableName()))); + List schemas = metadata.listSchemaNames(session); + assertEquals(ImmutableList.copyOf(schemas), ImmutableList.of("default")); + PinotTableHandle withWeirdSchema = metadata.getTableHandle(session, new SchemaTableName("foo", TestPinotSplitManager.realtimeOnlyTable.getTableName())); + assertEquals(withWeirdSchema.getTableName(), TestPinotSplitManager.realtimeOnlyTable.getTableName()); + PinotTableHandle withAnotherSchema = metadata.getTableHandle(session, new SchemaTableName(TestPinotSplitManager.realtimeOnlyTable.getTableName(), TestPinotSplitManager.realtimeOnlyTable.getTableName())); + assertEquals(withAnotherSchema.getTableName(), TestPinotSplitManager.realtimeOnlyTable.getTableName()); + } +} diff --git a/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotQueryBase.java b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotQueryBase.java new file mode 100644 index 0000000000000..ea01e0c9c1ba5 --- /dev/null +++ b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotQueryBase.java @@ -0,0 +1,225 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.pinot; + +import com.facebook.presto.Session; +import com.facebook.presto.SystemSessionProperties; +import com.facebook.presto.block.BlockEncodingManager; +import com.facebook.presto.execution.warnings.WarningCollector; +import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.Metadata; +import com.facebook.presto.metadata.MetadataManager; +import com.facebook.presto.metadata.SessionPropertyManager; +import com.facebook.presto.pinot.query.PinotQueryGeneratorContext; +import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.ConnectorId; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.TableHandle; +import com.facebook.presto.spi.block.SortOrder; +import com.facebook.presto.spi.function.StandardFunctionResolution; +import com.facebook.presto.spi.plan.Assignments; +import com.facebook.presto.spi.plan.FilterNode; +import com.facebook.presto.spi.plan.LimitNode; +import com.facebook.presto.spi.plan.Ordering; +import com.facebook.presto.spi.plan.OrderingScheme; +import com.facebook.presto.spi.plan.PlanNode; +import com.facebook.presto.spi.plan.PlanNodeIdAllocator; +import com.facebook.presto.spi.plan.ProjectNode; +import com.facebook.presto.spi.plan.TableScanNode; +import com.facebook.presto.spi.plan.TopNNode; +import com.facebook.presto.spi.relation.RowExpression; +import com.facebook.presto.spi.relation.VariableReferenceExpression; +import com.facebook.presto.spi.type.Type; +import com.facebook.presto.spi.type.TypeManager; +import com.facebook.presto.sql.ExpressionUtils; +import com.facebook.presto.sql.analyzer.FeaturesConfig; +import com.facebook.presto.sql.parser.ParsingOptions; +import com.facebook.presto.sql.parser.SqlParser; +import com.facebook.presto.sql.planner.TypeProvider; +import com.facebook.presto.sql.planner.iterative.rule.test.PlanBuilder; +import com.facebook.presto.sql.relational.FunctionResolution; +import com.facebook.presto.sql.relational.SqlToRowExpressionTranslator; +import com.facebook.presto.sql.tree.Expression; +import com.facebook.presto.sql.tree.NodeRef; +import com.facebook.presto.testing.TestingConnectorSession; +import com.facebook.presto.testing.TestingSession; +import com.facebook.presto.testing.TestingTransactionHandle; +import com.facebook.presto.type.TypeRegistry; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; + +import java.util.Arrays; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.IntStream; + +import static com.facebook.presto.pinot.PinotColumnHandle.PinotColumnType.REGULAR; +import static com.facebook.presto.pinot.query.PinotQueryGeneratorContext.Origin.DERIVED; +import static com.facebook.presto.pinot.query.PinotQueryGeneratorContext.Origin.TABLE_COLUMN; +import static com.facebook.presto.spi.plan.LimitNode.Step.FINAL; +import static com.facebook.presto.spi.type.BigintType.BIGINT; +import static com.facebook.presto.spi.type.DoubleType.DOUBLE; +import static com.facebook.presto.spi.type.VarcharType.VARCHAR; +import static com.facebook.presto.sql.analyzer.ExpressionAnalyzer.getExpressionTypes; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static java.util.Locale.ENGLISH; +import static java.util.Objects.requireNonNull; +import static java.util.function.Function.identity; +import static java.util.stream.Collectors.toMap; + +public class TestPinotQueryBase +{ + protected static final TypeManager typeManager = new TypeRegistry(); + protected static final FunctionManager functionMetadataManager = new FunctionManager(typeManager, new BlockEncodingManager(typeManager), new FeaturesConfig()); + protected static final StandardFunctionResolution standardFunctionResolution = new FunctionResolution(functionMetadataManager); + + protected static ConnectorId pinotConnectorId = new ConnectorId("id"); + protected static PinotTableHandle realtimeOnlyTable = new PinotTableHandle(pinotConnectorId.getCatalogName(), "schema", "realtimeOnly"); + protected static PinotTableHandle hybridTable = new PinotTableHandle(pinotConnectorId.getCatalogName(), "schema", "hybrid"); + protected static PinotColumnHandle regionId = new PinotColumnHandle("regionId", BIGINT, REGULAR); + protected static PinotColumnHandle city = new PinotColumnHandle("city", VARCHAR, REGULAR); + protected static final PinotColumnHandle fare = new PinotColumnHandle("fare", DOUBLE, REGULAR); + protected static final PinotColumnHandle secondsSinceEpoch = new PinotColumnHandle("secondsSinceEpoch", BIGINT, REGULAR); + + protected static final Metadata metadata = MetadataManager.createTestMetadataManager(); + + protected final PinotConfig pinotConfig = new PinotConfig(); + + protected static final Map testInput = ImmutableMap.of( + new VariableReferenceExpression("regionid", BIGINT), new PinotQueryGeneratorContext.Selection("regionId", TABLE_COLUMN), // direct column reference + new VariableReferenceExpression("city", VARCHAR), new PinotQueryGeneratorContext.Selection("city", TABLE_COLUMN), // direct column reference + new VariableReferenceExpression("fare", DOUBLE), new PinotQueryGeneratorContext.Selection("fare", TABLE_COLUMN), // direct column reference + new VariableReferenceExpression("totalfare", DOUBLE), new PinotQueryGeneratorContext.Selection("(fare + trip)", DERIVED), // derived column + new VariableReferenceExpression("secondssinceepoch", BIGINT), new PinotQueryGeneratorContext.Selection("secondsSinceEpoch", TABLE_COLUMN)); // column for datetime functions + + protected final TypeProvider typeProvider = TypeProvider.fromVariables(testInput.keySet()); + + protected static class SessionHolder + { + private final ConnectorSession connectorSession; + private final Session session; + + public SessionHolder(PinotConfig pinotConfig) + { + connectorSession = new TestingConnectorSession(new PinotSessionProperties(pinotConfig).getSessionProperties()); + session = TestingSession.testSessionBuilder(new SessionPropertyManager(new SystemSessionProperties().getSessionProperties())).build(); + } + + public SessionHolder(boolean useDateTrunc) + { + this(new PinotConfig().setUseDateTrunc(useDateTrunc)); + } + + public ConnectorSession getConnectorSession() + { + return connectorSession; + } + + public Session getSession() + { + return session; + } + } + + protected VariableReferenceExpression v(String name) + { + return testInput.keySet().stream().filter(v -> v.getName().equals(name)).findFirst().orElseThrow(() -> new IllegalArgumentException("Cannot find variable " + name)); + } + + protected TableScanNode tableScan(PlanBuilder planBuilder, PinotTableHandle connectorTableHandle, PinotColumnHandle... columnHandles) + { + List variables = Arrays.stream(columnHandles).map(ch -> new VariableReferenceExpression(ch.getColumnName().toLowerCase(ENGLISH), ch.getDataType())).collect(toImmutableList()); + ImmutableMap.Builder assignments = ImmutableMap.builder(); + for (int i = 0; i < variables.size(); ++i) { + assignments.put(variables.get(i), columnHandles[i]); + } + TableHandle tableHandle = new TableHandle( + pinotConnectorId, + connectorTableHandle, + TestingTransactionHandle.create(), + Optional.empty()); + return planBuilder.tableScan( + tableHandle, + variables, + assignments.build()); + } + + protected FilterNode filter(PlanBuilder planBuilder, PlanNode source, RowExpression predicate) + { + return planBuilder.filter(predicate, source); + } + + protected ProjectNode project(PlanBuilder planBuilder, PlanNode source, List columnNames) + { + Map incomingColumns = source.getOutputVariables().stream().collect(toMap(VariableReferenceExpression::getName, identity())); + Assignments.Builder assignmentsBuilder = Assignments.builder(); + columnNames.forEach(columnName -> { + VariableReferenceExpression variable = requireNonNull(incomingColumns.get(columnName), "Couldn't find the incoming column " + columnName); + assignmentsBuilder.put(variable, variable); + }); + return planBuilder.project(assignmentsBuilder.build(), source); + } + + protected ProjectNode project(PlanBuilder planBuilder, PlanNode source, LinkedHashMap toProject, SessionHolder sessionHolder) + { + Assignments.Builder assignmentsBuilder = Assignments.builder(); + toProject.forEach((columnName, expression) -> { + RowExpression rowExpression = getRowExpression(expression, sessionHolder); + VariableReferenceExpression variable = new VariableReferenceExpression(columnName, rowExpression.getType()); + assignmentsBuilder.put(variable, rowExpression); + }); + return planBuilder.project(assignmentsBuilder.build(), source); + } + + public static Expression expression(String sql) + { + return ExpressionUtils.rewriteIdentifiersToSymbolReferences(new SqlParser().createExpression(sql, new ParsingOptions(ParsingOptions.DecimalLiteralTreatment.AS_DECIMAL))); + } + + protected RowExpression toRowExpression(Expression expression, Session session) + { + Map, Type> expressionTypes = getExpressionTypes( + session, + metadata, + new SqlParser(), + typeProvider, + expression, + ImmutableList.of(), + WarningCollector.NOOP); + return SqlToRowExpressionTranslator.translate(expression, expressionTypes, ImmutableMap.of(), functionMetadataManager, typeManager, session); + } + + protected LimitNode limit(PlanBuilder pb, long count, PlanNode source) + { + return new LimitNode(pb.getIdAllocator().getNextId(), source, count, FINAL); + } + + protected TopNNode topN(PlanBuilder pb, long count, List orderingColumns, List ascending, PlanNode source) + { + ImmutableList ordering = IntStream.range(0, orderingColumns.size()).boxed().map(i -> new Ordering(v(orderingColumns.get(i)), ascending.get(i) ? SortOrder.ASC_NULLS_FIRST : SortOrder.DESC_NULLS_FIRST)).collect(toImmutableList()); + return new TopNNode(pb.getIdAllocator().getNextId(), source, count, new OrderingScheme(ordering), TopNNode.Step.SINGLE); + } + + protected RowExpression getRowExpression(String sqlExpression, SessionHolder sessionHolder) + { + return toRowExpression(expression(sqlExpression), sessionHolder.getSession()); + } + + protected PlanBuilder createPlanBuilder(SessionHolder sessionHolder) + { + return new PlanBuilder(sessionHolder.getSession(), new PlanNodeIdAllocator(), metadata); + } +} diff --git a/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotSessionProperties.java b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotSessionProperties.java new file mode 100644 index 0000000000000..7de528cbb67bd --- /dev/null +++ b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotSessionProperties.java @@ -0,0 +1,42 @@ +/* + * 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.pinot; + +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.testing.TestingConnectorSession; +import io.airlift.units.Duration; +import org.testng.annotations.Test; + +import java.util.concurrent.TimeUnit; + +import static com.facebook.presto.testing.assertions.Assert.assertEquals; + +public class TestPinotSessionProperties +{ + @Test(expectedExceptions = IllegalArgumentException.class) + public void testInvalidNumSegmentSplits() + { + new PinotConfig().setNumSegmentsPerSplit(-3); + } + + @Test + public void testConnectionTimeoutParsedProperly() + { + PinotConfig pinotConfig = new PinotConfig().setConnectionTimeout(new Duration(15, TimeUnit.SECONDS)); + PinotSessionProperties pinotSessionProperties = new PinotSessionProperties(pinotConfig); + ConnectorSession session = new TestingConnectorSession(pinotSessionProperties.getSessionProperties()); + assertEquals(PinotSessionProperties.getConnectionTimeout(session), new Duration(0.25, TimeUnit.MINUTES)); + } +} diff --git a/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotSplitManager.java b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotSplitManager.java new file mode 100644 index 0000000000000..f52aac2aff924 --- /dev/null +++ b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotSplitManager.java @@ -0,0 +1,165 @@ +/* + * 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.pinot; + +import com.facebook.presto.pinot.query.PinotQueryGenerator; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.ConnectorSplitSource; +import com.facebook.presto.spi.plan.PlanNode; +import com.facebook.presto.sql.analyzer.FeaturesConfig; +import com.facebook.presto.sql.planner.iterative.rule.test.PlanBuilder; +import com.facebook.presto.testing.TestingConnectorSession; +import com.google.common.base.Splitter; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.testng.annotations.Test; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.Executors; + +import static com.facebook.airlift.concurrent.MoreFutures.getFutureValue; +import static com.facebook.presto.pinot.PinotSplit.SplitType.BROKER; +import static com.facebook.presto.pinot.PinotSplit.SplitType.SEGMENT; +import static com.facebook.presto.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; +import static com.facebook.presto.spi.type.TimeZoneKey.UTC_KEY; +import static java.util.Locale.ENGLISH; +import static java.util.stream.Collectors.toList; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; + +public class TestPinotSplitManager + extends TestPinotQueryBase +{ + // Test table and related info + private final PinotConfig pinotConfig = new PinotConfig(); + private final PinotConnection pinotConnection = new PinotConnection(new MockPinotClusterInfoFetcher(pinotConfig), pinotConfig, Executors.newSingleThreadExecutor()); + private final PinotSplitManager pinotSplitManager = new PinotSplitManager(pinotConnectorId, pinotConnection); + + @Test + public void testRealtimeSegmentSplitsOneSegmentPerServer() + { + testSegmentSplitsHelperNoFilter(realtimeOnlyTable, 1, 4, false); // 2 servers with 2 segments each + } + + private void testSegmentSplitsHelperNoFilter(PinotTableHandle table, int segmentsPerSplit, int expectedNumSplits, boolean expectFilter) + { + PinotConfig pinotConfig = new PinotConfig().setPreferBrokerQueries(false); + SessionHolder sessionHolder = new SessionHolder(pinotConfig); + PlanBuilder planBuilder = createPlanBuilder(sessionHolder); + PlanNode plan = tableScan(planBuilder, table, regionId, city, fare, secondsSinceEpoch); + PinotQueryGenerator.GeneratedPql generatedPql = new PinotQueryGenerator(pinotConfig, typeManager, functionMetadataManager, standardFunctionResolution).generate(plan, sessionHolder.getConnectorSession()).get().getGeneratedPql(); + PinotTableHandle pinotTableHandle = new PinotTableHandle(table.getConnectorId(), table.getSchemaName(), table.getTableName(), Optional.of(false), Optional.of(generatedPql)); + List splits = getSplitsHelper(pinotTableHandle, segmentsPerSplit, false); + assertSplits(splits, expectedNumSplits, SEGMENT); + splits.forEach(s -> assertSegmentSplitWellFormed(s, expectFilter)); + } + + private void testSegmentSplitsHelperWithFilter(PinotTableHandle table, int segmentsPerSplit, int expectedNumSplits) + { + PinotConfig pinotConfig = new PinotConfig().setPreferBrokerQueries(false); + SessionHolder sessionHolder = new SessionHolder(pinotConfig); + PlanBuilder planBuilder = createPlanBuilder(sessionHolder); + PlanNode plan = filter(planBuilder, tableScan(planBuilder, table, regionId, city, fare, secondsSinceEpoch), getRowExpression("city = 'Boston'", sessionHolder)); + PinotQueryGenerator.GeneratedPql generatedPql = new PinotQueryGenerator(pinotConfig, typeManager, functionMetadataManager, standardFunctionResolution).generate(plan, sessionHolder.getConnectorSession()).get().getGeneratedPql(); + PinotTableHandle pinotTableHandle = new PinotTableHandle(table.getConnectorId(), table.getSchemaName(), table.getTableName(), Optional.of(false), Optional.of(generatedPql)); + List splits = getSplitsHelper(pinotTableHandle, segmentsPerSplit, false); + assertSplits(splits, expectedNumSplits, SEGMENT); + splits.forEach(s -> assertSegmentSplitWellFormed(s, true)); + } + + @Test + public void testSplitsBroker() + { + PinotQueryGenerator.GeneratedPql generatedPql = new PinotQueryGenerator.GeneratedPql(realtimeOnlyTable.getTableName(), String.format("SELECT %s, COUNT(1) FROM %s GROUP BY %s TOP %d", city.getColumnName(), realtimeOnlyTable.getTableName(), city.getColumnName(), pinotConfig.getTopNLarge()), ImmutableList.of(0, 1), 1, false, true); + PinotTableHandle pinotTableHandle = new PinotTableHandle(realtimeOnlyTable.getConnectorId(), realtimeOnlyTable.getSchemaName(), realtimeOnlyTable.getTableName(), Optional.of(true), Optional.of(generatedPql)); + List splits = getSplitsHelper(pinotTableHandle, 1, false); + assertSplits(splits, 1, BROKER); + } + + @Test(expectedExceptions = PinotSplitManager.QueryNotAdequatelyPushedDownException.class) + public void testBrokerNonShortQuery() + { + PinotQueryGenerator.GeneratedPql generatedPql = new PinotQueryGenerator.GeneratedPql(realtimeOnlyTable.getTableName(), String.format("SELECT %s FROM %s", city.getColumnName(), realtimeOnlyTable.getTableName()), ImmutableList.of(0), 0, false, false); + PinotTableHandle pinotTableHandle = new PinotTableHandle(realtimeOnlyTable.getConnectorId(), realtimeOnlyTable.getSchemaName(), realtimeOnlyTable.getTableName(), Optional.of(false), Optional.of(generatedPql)); + List splits = getSplitsHelper(pinotTableHandle, 1, true); + assertSplits(splits, 1, BROKER); + } + + @Test + public void testRealtimeSegmentSplitsManySegmentPerServer() + { + testSegmentSplitsHelperNoFilter(realtimeOnlyTable, Integer.MAX_VALUE, 2, false); + } + + @Test + public void testHybridSegmentSplitsOneSegmentPerServer() + { + testSegmentSplitsHelperNoFilter(hybridTable, 1, 8, true); + testSegmentSplitsHelperWithFilter(hybridTable, 1, 8); + } + + private void assertSplits(List splits, int numSplitsExpected, PinotSplit.SplitType splitType) + { + assertEquals(splits.size(), numSplitsExpected); + splits.forEach(s -> assertEquals(s.getSplitType(), splitType)); + } + + private void assertSegmentSplitWellFormed(PinotSplit split, boolean expectFilter) + { + assertEquals(split.getSplitType(), SEGMENT); + assertTrue(split.getSegmentPql().isPresent()); + assertTrue(split.getSegmentHost().isPresent()); + assertFalse(split.getSegments().isEmpty()); + String pql = split.getSegmentPql().get(); + assertFalse(pql.contains("__")); // templates should be fully resolved + List splitOnWhere = Splitter.on(" WHERE ").splitToList(pql); + // There should be exactly one WHERE clause and it should partition the pql into two + assertEquals(splitOnWhere.size(), expectFilter ? 2 : 1, "Expected to find only one WHERE clause in " + pql); + } + + public static ConnectorSession createSessionWithNumSplits(int numSegmentsPerSplit, boolean forbidSegmentQueries, PinotConfig pinotConfig) + { + return new TestingConnectorSession( + "user", + Optional.of("test"), + Optional.empty(), + UTC_KEY, + ENGLISH, + System.currentTimeMillis(), + new PinotSessionProperties(pinotConfig).getSessionProperties(), + ImmutableMap.of( + PinotSessionProperties.NUM_SEGMENTS_PER_SPLIT, + numSegmentsPerSplit, + PinotSessionProperties.FORBID_SEGMENT_QUERIES, + forbidSegmentQueries), + new FeaturesConfig().isLegacyTimestamp(), + Optional.empty()); + } + + private List getSplitsHelper(PinotTableHandle pinotTable, int numSegmentsPerSplit, boolean forbidSegmentQueries) + { + PinotTableLayoutHandle pinotTableLayout = new PinotTableLayoutHandle(pinotTable); + ConnectorSession session = createSessionWithNumSplits(numSegmentsPerSplit, forbidSegmentQueries, pinotConfig); + ConnectorSplitSource splitSource = pinotSplitManager.getSplits(null, session, pinotTableLayout, null); + List splits = new ArrayList<>(); + while (!splitSource.isFinished()) { + splits.addAll(getFutureValue(splitSource.getNextBatch(NOT_PARTITIONED, 1000)).getSplits().stream().map(s -> (PinotSplit) s).collect(toList())); + } + + return splits; + } +} diff --git a/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotTableHandle.java b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotTableHandle.java new file mode 100644 index 0000000000000..054097694c0c8 --- /dev/null +++ b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotTableHandle.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.pinot; + +import com.facebook.airlift.json.JsonCodec; +import com.facebook.airlift.testing.EquivalenceTester; +import org.testng.annotations.Test; + +import static com.facebook.airlift.json.JsonCodec.jsonCodec; +import static org.testng.Assert.assertEquals; + +public class TestPinotTableHandle +{ + private final PinotTableHandle tableHandle = new PinotTableHandle("connectorId", "schemaName", "tableName"); + + @Test + public void testJsonRoundTrip() + { + JsonCodec codec = jsonCodec(PinotTableHandle.class); + String json = codec.toJson(tableHandle); + PinotTableHandle copy = codec.fromJson(json); + assertEquals(copy, tableHandle); + } + + @Test + public void testEquivalence() + { + EquivalenceTester.equivalenceTester() + .addEquivalentGroup( + new PinotTableHandle("connector", "schema", "table"), + new PinotTableHandle("connector", "schema", "table")) + .addEquivalentGroup( + new PinotTableHandle("connectorX", "schema", "table"), + new PinotTableHandle("connectorX", "schema", "table")) + .addEquivalentGroup( + new PinotTableHandle("connector", "schemaX", "table"), + new PinotTableHandle("connector", "schemaX", "table")) + .addEquivalentGroup( + new PinotTableHandle("connector", "schema", "tableX"), + new PinotTableHandle("connector", "schema", "tableX")) + .check(); + } +} diff --git a/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/query/TestPinotConnectorPlanOptimizer.java b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/query/TestPinotConnectorPlanOptimizer.java new file mode 100644 index 0000000000000..70ef43fb39bb8 --- /dev/null +++ b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/query/TestPinotConnectorPlanOptimizer.java @@ -0,0 +1,233 @@ +/* + * 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.pinot.query; + +import com.facebook.presto.Session; +import com.facebook.presto.cost.PlanNodeStatsEstimate; +import com.facebook.presto.cost.StatsAndCosts; +import com.facebook.presto.cost.StatsProvider; +import com.facebook.presto.expressions.LogicalRowExpressions; +import com.facebook.presto.metadata.Metadata; +import com.facebook.presto.pinot.PinotConfig; +import com.facebook.presto.pinot.PinotConnectorPlanOptimizer; +import com.facebook.presto.pinot.PinotTableHandle; +import com.facebook.presto.pinot.TestPinotQueryBase; +import com.facebook.presto.pinot.TestPinotSplitManager; +import com.facebook.presto.spi.ConnectorId; +import com.facebook.presto.spi.plan.FilterNode; +import com.facebook.presto.spi.plan.PlanNode; +import com.facebook.presto.spi.plan.TableScanNode; +import com.facebook.presto.spi.relation.VariableReferenceExpression; +import com.facebook.presto.sql.planner.Plan; +import com.facebook.presto.sql.planner.PlanVariableAllocator; +import com.facebook.presto.sql.planner.TypeProvider; +import com.facebook.presto.sql.planner.assertions.ExpectedValueProvider; +import com.facebook.presto.sql.planner.assertions.MatchResult; +import com.facebook.presto.sql.planner.assertions.Matcher; +import com.facebook.presto.sql.planner.assertions.PlanAssert; +import com.facebook.presto.sql.planner.assertions.PlanMatchPattern; +import com.facebook.presto.sql.planner.assertions.SymbolAliases; +import com.facebook.presto.sql.planner.iterative.rule.test.PlanBuilder; +import com.facebook.presto.sql.relational.FunctionResolution; +import com.facebook.presto.sql.relational.RowExpressionDeterminismEvaluator; +import com.facebook.presto.sql.tree.FunctionCall; +import com.facebook.presto.sql.tree.SymbolReference; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.testng.annotations.Test; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.regex.Pattern; + +import static com.facebook.presto.spi.type.BigintType.BIGINT; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.aggregation; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.node; +import static com.google.common.base.MoreObjects.toStringHelper; +import static com.google.common.base.Preconditions.checkState; +import static java.util.function.Function.identity; +import static java.util.stream.Collectors.toMap; + +public class TestPinotConnectorPlanOptimizer + extends TestPinotQueryBase +{ + private static final SessionHolder defaultSessionHolder = new SessionHolder(false); + private final LogicalRowExpressions logicalRowExpressions = new LogicalRowExpressions( + new RowExpressionDeterminismEvaluator(functionMetadataManager), + new FunctionResolution(functionMetadataManager), + functionMetadataManager); + private final PinotTableHandle pinotTable = TestPinotSplitManager.hybridTable; + + private static void assertPlanMatch(PlanNode actual, PlanMatchPattern expected, TypeProvider typeProvider) + { + PlanAssert.assertPlan( + defaultSessionHolder.getSession(), + metadata, + (node, sourceStats, lookup, session, types) -> PlanNodeStatsEstimate.unknown(), + new Plan(actual, typeProvider, StatsAndCosts.empty()), + expected); + } + + private static final class PinotTableScanMatcher + implements Matcher + { + private final ConnectorId connectorId; + private final String tableName; + private final Optional pqlRegex; + private final Optional scanParallelismExpected; + private final String[] columns; + + static PlanMatchPattern match( + String connectorName, + String tableName, + Optional pqlRegex, + Optional scanParallelismExpected, + String... columnNames) + { + return node(TableScanNode.class) + .with(new PinotTableScanMatcher( + new ConnectorId(connectorName), + tableName, + pqlRegex, scanParallelismExpected, columnNames)); + } + + static PlanMatchPattern match( + PinotTableHandle tableHandle, + Optional pqlRegex, + Optional scanParallelismExpected, + List variables) + { + return match(tableHandle.getConnectorId(), + tableHandle.getTableName(), + pqlRegex, + scanParallelismExpected, + variables.stream().map(VariableReferenceExpression::getName).toArray(String[]::new)); + } + + private PinotTableScanMatcher( + ConnectorId connectorId, + String tableName, + Optional pqlRegex, + Optional scanParallelismExpected, + String... columns) + { + this.connectorId = connectorId; + this.pqlRegex = pqlRegex; + this.scanParallelismExpected = scanParallelismExpected; + this.columns = columns; + this.tableName = tableName; + } + + @Override + public boolean shapeMatches(PlanNode node) + { + return node instanceof TableScanNode; + } + + private static boolean checkPqlMatches(Optional regex, Optional pql) + { + if (!pql.isPresent() && !regex.isPresent()) { + return true; + } + if (pql.isPresent() && regex.isPresent()) { + String toMatch = pql.get(); + Pattern compiled = Pattern.compile(regex.get(), Pattern.CASE_INSENSITIVE); + return compiled.matcher(toMatch).matches(); + } + return false; + } + + @Override + public MatchResult detailMatches( + PlanNode node, + StatsProvider stats, + Session session, + Metadata metadata, + SymbolAliases symbolAliases) + { + checkState(shapeMatches(node), "Plan testing framework error: shapeMatches returned false in detailMatches in %s", this.getClass().getName()); + + TableScanNode tableScanNode = (TableScanNode) node; + if (connectorId.equals(tableScanNode.getTable().getConnectorId())) { + PinotTableHandle pinotTableHandle = (PinotTableHandle) tableScanNode.getTable().getConnectorHandle(); + if (pinotTableHandle.getTableName().equals(tableName)) { + Optional pql = pinotTableHandle.getPql().map(PinotQueryGenerator.GeneratedPql::getPql); + if (checkPqlMatches(pqlRegex, pql)) { + return MatchResult.match(SymbolAliases.builder().putAll(Arrays.stream(columns).collect(toMap(identity(), SymbolReference::new))).build()); + } + } + } + return MatchResult.NO_MATCH; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("connectorId", connectorId) + .add("tableName", tableName) + .add("pqlRegex", pqlRegex) + .add("scanParallelismExpected", scanParallelismExpected) + .add("columns", columns) + .toString(); + } + } + + @Test + public void testLimitPushdownWithStarSelection() + { + PlanBuilder pb = createPlanBuilder(defaultSessionHolder); + PlanNode originalPlan = limit(pb, 50L, tableScan(pb, pinotTable, regionId, city, fare, secondsSinceEpoch)); + PlanNode optimized = getOptimizedPlan(pb, originalPlan, true); + assertPlanMatch(optimized, PinotTableScanMatcher.match(pinotTable, Optional.of("SELECT regionId, city, fare, secondsSinceEpoch FROM hybrid LIMIT 50"), Optional.of(false), originalPlan.getOutputVariables()), typeProvider); + } + + @Test + public void testPartialPredicatePushdown() + { + PlanBuilder pb = createPlanBuilder(defaultSessionHolder); + TableScanNode tableScanNode = tableScan(pb, pinotTable, regionId, city, fare, secondsSinceEpoch); + FilterNode filter = filter(pb, tableScanNode, getRowExpression("lower(substr(city, 0, 3)) = 'del' AND fare > 100", defaultSessionHolder)); + PlanNode originalPlan = limit(pb, 50L, filter); + PlanNode optimized = getOptimizedPlan(pb, originalPlan, true); + PlanMatchPattern tableScanMatcher = PinotTableScanMatcher.match(pinotTable, Optional.of("SELECT regionId, city, fare, secondsSinceEpoch FROM hybrid__TABLE_NAME_SUFFIX_TEMPLATE__ WHERE \\(fare > 100\\).*"), Optional.of(true), filter.getOutputVariables()); + assertPlanMatch(optimized, PlanMatchPattern.limit(50L, PlanMatchPattern.filter("lower(substr(city, 0, 3)) = 'del'", tableScanMatcher)), typeProvider); + } + + @Test + public void testUnsupportedPredicatePushdown() + { + Map> aggregationsSecond = ImmutableMap.of( + "count", PlanMatchPattern.functionCall("count", false, ImmutableList.of())); + + PlanBuilder planBuilder = createPlanBuilder(defaultSessionHolder); + PlanNode limit = limit(planBuilder, 50L, tableScan(planBuilder, pinotTable, regionId, city, fare, secondsSinceEpoch)); + PlanNode originalPlan = planBuilder.aggregation(builder -> builder.source(limit).globalGrouping().addAggregation(new VariableReferenceExpression("count", BIGINT), getRowExpression("count(*)", defaultSessionHolder))); + + PlanNode optimized = getOptimizedPlan(planBuilder, originalPlan, true); + + PlanMatchPattern tableScanMatcher = PinotTableScanMatcher.match(pinotTable, Optional.of("SELECT regionId, city, fare, secondsSinceEpoch FROM hybrid LIMIT 50"), Optional.of(false), originalPlan.getOutputVariables()); + assertPlanMatch(optimized, aggregation(aggregationsSecond, tableScanMatcher), typeProvider); + } + + private PlanNode getOptimizedPlan(PlanBuilder planBuilder, PlanNode originalPlan, boolean scanParallelism) + { + PinotConfig pinotConfig = new PinotConfig().setPreferBrokerQueries(scanParallelism); + PinotQueryGenerator pinotQueryGenerator = new PinotQueryGenerator(pinotConfig, typeManager, functionMetadataManager, standardFunctionResolution); + PinotConnectorPlanOptimizer optimizer = new PinotConnectorPlanOptimizer(pinotQueryGenerator, typeManager, functionMetadataManager, logicalRowExpressions, standardFunctionResolution); + return optimizer.optimize(originalPlan, defaultSessionHolder.getConnectorSession(), new PlanVariableAllocator(), planBuilder.getIdAllocator()); + } +} diff --git a/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/query/TestPinotExpressionConverters.java b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/query/TestPinotExpressionConverters.java new file mode 100644 index 0000000000000..52d474b11ffc4 --- /dev/null +++ b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/query/TestPinotExpressionConverters.java @@ -0,0 +1,171 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.pinot.query; + +import com.facebook.presto.pinot.PinotException; +import com.facebook.presto.pinot.TestPinotQueryBase; +import com.facebook.presto.spi.relation.RowExpression; +import com.facebook.presto.spi.relation.VariableReferenceExpression; +import org.testng.annotations.Test; + +import java.util.function.Function; + +import static com.facebook.presto.pinot.PinotErrorCode.PINOT_UNSUPPORTED_EXPRESSION; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.fail; + +public class TestPinotExpressionConverters + extends TestPinotQueryBase +{ + private final Function testInputFunction = testInput::get; + + @Test + public void testProjectExpressionConverter() + { + SessionHolder sessionHolder = new SessionHolder(false); + testProject("secondssinceepoch", "secondsSinceEpoch", sessionHolder); + // functions + testAggregationProject("date_trunc('hour', from_unixtime(secondssinceepoch))", + "dateTimeConvert(secondsSinceEpoch, '1:SECONDS:EPOCH', '1:MILLISECONDS:EPOCH', '1:HOURS')", sessionHolder); + + // arithmetic + testAggregationProject("regionid + 1", "ADD(regionId, 1)", sessionHolder); + testAggregationProject("regionid - 1", "SUB(regionId, 1)", sessionHolder); + testAggregationProject("1 * regionid", "MULT(1, regionId)", sessionHolder); + testAggregationProject("1 / regionid", "DIV(1, regionId)", sessionHolder); + + // TODO ... this one is failing + testAggregationProject("secondssinceepoch + 1559978258.674", "ADD(secondsSinceEpoch, 1559978258.674)", sessionHolder); + + testAggregationProject("secondssinceepoch + 1559978258", "ADD(secondsSinceEpoch, 1559978258)", sessionHolder); + + testAggregationProjectUnsupported("secondssinceepoch > 0", sessionHolder); + + testAggregationProject("date_trunc('hour', from_unixtime(secondssinceepoch + 2))", + "dateTimeConvert(ADD(secondsSinceEpoch, 2), '1:SECONDS:EPOCH', '1:MILLISECONDS:EPOCH', '1:HOURS')", sessionHolder); + } + + private void testProject(String sqlExpression, String expectedPinotExpression, SessionHolder sessionHolder) + { + RowExpression pushDownExpression = getRowExpression(sqlExpression, sessionHolder); + String actualPinotExpression = pushDownExpression.accept(new PinotProjectExpressionConverter( + typeManager, + standardFunctionResolution), + testInput).getDefinition(); + assertEquals(actualPinotExpression, expectedPinotExpression); + } + + @Test + public void testAdhoc() + { + SessionHolder sessionHolder = new SessionHolder(false); + + testAggregationProject("secondssinceepoch + 1559978258.674", "ADD(secondsSinceEpoch, 1559978258.674)", sessionHolder); + } + + @Test + public void testDateTruncationConversion() + { + SessionHolder sessionHolder = new SessionHolder(true); + testAggregationProject("date_trunc('hour', from_unixtime(secondssinceepoch + 2))", + "dateTrunc(ADD(secondsSinceEpoch, 2),seconds, UTC, hour)", sessionHolder); + + testAggregationProject("date_trunc('hour', from_unixtime(secondssinceepoch + 2, 'America/New_York'))", + "dateTrunc(ADD(secondsSinceEpoch, 2),seconds, America/New_York, hour)", sessionHolder); + } + + @Test + public void testFilterExpressionConverter() + { + SessionHolder sessionHolder = new SessionHolder(false); + + // Simple comparisons + testFilter("regionid = 20", "(regionId = 20)", sessionHolder); + testFilter("regionid >= 20", "(regionId >= 20)", sessionHolder); + testFilter("city = 'Campbell'", "(city = 'Campbell')", sessionHolder); + + // between + testFilter("totalfare between 20 and 30", "((fare + trip) BETWEEN 20 AND 30)", sessionHolder); + + // in, not in + testFilter("regionid in (20, 30, 40)", "(regionId IN (20, 30, 40))", sessionHolder); + testFilter("regionid not in (20, 30, 40)", "(regionId NOT IN (20, 30, 40))", sessionHolder); + testFilter("city in ('San Jose', 'Campbell', 'Union City')", "(city IN ('San Jose', 'Campbell', 'Union City'))", sessionHolder); + testFilter("city not in ('San Jose', 'Campbell', 'Union City')", "(city NOT IN ('San Jose', 'Campbell', 'Union City'))", sessionHolder); + testFilterUnsupported("secondssinceepoch + 1 in (234, 24324)", sessionHolder); + testFilterUnsupported("NOT (secondssinceepoch = 2323)", sessionHolder); + + // combinations + testFilter("totalfare between 20 and 30 AND regionid > 20 OR city = 'Campbell'", + "((((fare + trip) BETWEEN 20 AND 30) AND (regionId > 20)) OR (city = 'Campbell'))", sessionHolder); + + testFilter("secondssinceepoch > 1559978258", "(secondsSinceEpoch > 1559978258)", sessionHolder); + } + + private void testAggregationProject(String sqlExpression, String expectedPinotExpression, SessionHolder sessionHolder) + { + RowExpression pushDownExpression = getRowExpression(sqlExpression, sessionHolder); + String actualPinotExpression = pushDownExpression.accept(new PinotAggregationProjectConverter( + typeManager, + functionMetadataManager, + standardFunctionResolution, + sessionHolder.getConnectorSession()), + testInput).getDefinition(); + assertEquals(actualPinotExpression, expectedPinotExpression); + } + + private void testAggregationProjectUnsupported(String sqlExpression, SessionHolder sessionHolder) + { + try { + RowExpression pushDownExpression = getRowExpression(sqlExpression, sessionHolder); + String actualPinotExpression = pushDownExpression.accept(new PinotAggregationProjectConverter( + typeManager, + functionMetadataManager, + standardFunctionResolution, + sessionHolder.getConnectorSession()), + testInput).getDefinition(); + fail("expected to not reach here: Generated " + actualPinotExpression); + } + catch (PinotException e) { + assertEquals(e.getErrorCode(), PINOT_UNSUPPORTED_EXPRESSION.toErrorCode()); + } + } + + private void testFilter(String sqlExpression, String expectedPinotExpression, SessionHolder sessionHolder) + { + RowExpression pushDownExpression = getRowExpression(sqlExpression, sessionHolder); + String actualPinotExpression = pushDownExpression.accept(new PinotFilterExpressionConverter( + typeManager, + functionMetadataManager, + standardFunctionResolution), + testInputFunction).getDefinition(); + assertEquals(actualPinotExpression, expectedPinotExpression); + } + + private void testFilterUnsupported(String sqlExpression, SessionHolder sessionHolder) + { + try { + RowExpression pushDownExpression = getRowExpression(sqlExpression, sessionHolder); + String actualPinotExpression = pushDownExpression.accept(new PinotFilterExpressionConverter( + typeManager, + functionMetadataManager, + standardFunctionResolution), + testInputFunction).getDefinition(); + fail("expected to not reach here: Generated " + actualPinotExpression); + } + catch (PinotException e) { + assertEquals(e.getErrorCode(), PINOT_UNSUPPORTED_EXPRESSION.toErrorCode()); + } + } +} diff --git a/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/query/TestPinotQueryGenerator.java b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/query/TestPinotQueryGenerator.java new file mode 100644 index 0000000000000..f203fd9750244 --- /dev/null +++ b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/query/TestPinotQueryGenerator.java @@ -0,0 +1,238 @@ +/* + * 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.pinot.query; + +import com.facebook.presto.pinot.PinotConfig; +import com.facebook.presto.pinot.PinotTableHandle; +import com.facebook.presto.pinot.TestPinotQueryBase; +import com.facebook.presto.spi.block.SortOrder; +import com.facebook.presto.spi.plan.AggregationNode; +import com.facebook.presto.spi.plan.Ordering; +import com.facebook.presto.spi.plan.OrderingScheme; +import com.facebook.presto.spi.plan.PlanNode; +import com.facebook.presto.spi.plan.TableScanNode; +import com.facebook.presto.spi.plan.TopNNode; +import com.facebook.presto.spi.relation.VariableReferenceExpression; +import com.facebook.presto.sql.planner.iterative.rule.test.PlanBuilder; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.testng.annotations.Test; + +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.function.BiConsumer; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static com.facebook.presto.spi.type.TimestampType.TIMESTAMP; +import static java.lang.String.format; +import static org.testng.Assert.assertEquals; + +public class TestPinotQueryGenerator + extends TestPinotQueryBase +{ + private static final SessionHolder defaultSessionHolder = new SessionHolder(false); + private static final PinotTableHandle pinotTable = realtimeOnlyTable; + + private void testPQL( + PinotConfig givenPinotConfig, + Function planBuilderConsumer, + String expectedPQL, SessionHolder sessionHolder, + Map outputVariables) + { + PlanNode planNode = planBuilderConsumer.apply(createPlanBuilder(sessionHolder)); + testPQL(givenPinotConfig, planNode, expectedPQL, sessionHolder, outputVariables); + } + + private void testPQL( + PinotConfig givenPinotConfig, + PlanNode planNode, + String expectedPQL, + SessionHolder sessionHolder, + Map outputVariables) + { + PinotQueryGenerator.PinotQueryGeneratorResult pinotQueryGeneratorResult = new PinotQueryGenerator(givenPinotConfig, typeManager, functionMetadataManager, standardFunctionResolution).generate(planNode, sessionHolder.getConnectorSession()).get(); + if (expectedPQL.contains("__expressions__")) { + String expressions = planNode.getOutputVariables().stream().map(v -> outputVariables.get(v.getName())).filter(v -> v != null).collect(Collectors.joining(", ")); + expectedPQL = expectedPQL.replace("__expressions__", expressions); + } + assertEquals(pinotQueryGeneratorResult.getGeneratedPql().getPql(), expectedPQL); + } + + private void testPQL(Function planBuilderConsumer, String expectedPQL, SessionHolder sessionHolder, Map outputVariables) + { + testPQL(pinotConfig, planBuilderConsumer, expectedPQL, sessionHolder, outputVariables); + } + + private void testPQL(Function planBuilderConsumer, String expectedPQL, SessionHolder sessionHolder) + { + testPQL(planBuilderConsumer, expectedPQL, sessionHolder, ImmutableMap.of()); + } + + private void testPQL(Function planBuilderConsumer, String expectedPQL) + { + testPQL(planBuilderConsumer, expectedPQL, defaultSessionHolder); + } + + private PlanNode buildPlan(Function consumer) + { + PlanBuilder planBuilder = createPlanBuilder(defaultSessionHolder); + return consumer.apply(planBuilder); + } + + private void testUnaryAggregationHelper(BiConsumer aggregationFunctionBuilder, String expectedAggOutput) + { + PlanNode justScan = buildPlan(planBuilder -> tableScan(planBuilder, pinotTable, regionId, secondsSinceEpoch, city, fare)); + PlanNode filter = buildPlan(planBuilder -> filter(planBuilder, tableScan(planBuilder, pinotTable, regionId, secondsSinceEpoch, city, fare), getRowExpression("fare > 3", defaultSessionHolder))); + PlanNode anotherFilter = buildPlan(planBuilder -> filter(planBuilder, tableScan(planBuilder, pinotTable, regionId, secondsSinceEpoch, city, fare), getRowExpression("secondssinceepoch between 200 and 300 and regionid >= 40", defaultSessionHolder))); + testPQL(planBuilder -> planBuilder.aggregation(aggBuilder -> aggregationFunctionBuilder.accept(planBuilder, aggBuilder.source(justScan).globalGrouping())), + format("SELECT %s FROM realtimeOnly", expectedAggOutput)); + testPQL(planBuilder -> planBuilder.aggregation(aggBuilder -> aggregationFunctionBuilder.accept(planBuilder, aggBuilder.source(filter).globalGrouping())), + format("SELECT %s FROM realtimeOnly WHERE (fare > 3)", expectedAggOutput)); + testPQL(planBuilder -> planBuilder.aggregation(aggBuilder -> aggregationFunctionBuilder.accept(planBuilder, aggBuilder.source(filter).singleGroupingSet(v("regionid")))), + format("SELECT %s FROM realtimeOnly WHERE (fare > 3) GROUP BY regionId TOP 10000", expectedAggOutput)); + testPQL(planBuilder -> planBuilder.aggregation(aggBuilder -> aggregationFunctionBuilder.accept(planBuilder, aggBuilder.source(justScan).singleGroupingSet(v("regionid")))), + format("SELECT %s FROM realtimeOnly GROUP BY regionId TOP 10000", expectedAggOutput)); + testPQL(planBuilder -> planBuilder.aggregation(aggBuilder -> aggregationFunctionBuilder.accept(planBuilder, aggBuilder.source(anotherFilter).singleGroupingSet(v("regionid"), v("city")))), + format("SELECT %s FROM realtimeOnly WHERE ((secondsSinceEpoch BETWEEN 200 AND 300) AND (regionId >= 40)) GROUP BY regionId, city TOP 10000", expectedAggOutput)); + } + + @Test + public void testSimpleSelectStar() + { + testPQL(planBuilder -> limit(planBuilder, 50L, tableScan(planBuilder, pinotTable, regionId, city, fare, secondsSinceEpoch)), + "SELECT regionId, city, fare, secondsSinceEpoch FROM realtimeOnly LIMIT 50"); + testPQL(planBuilder -> limit(planBuilder, 50L, tableScan(planBuilder, pinotTable, regionId, secondsSinceEpoch)), + "SELECT regionId, secondsSinceEpoch FROM realtimeOnly LIMIT 50"); + } + + @Test + public void testSimpleSelectWithFilterLimit() + { + testPQL(planBuilder -> limit(planBuilder, 50L, project(planBuilder, filter(planBuilder, tableScan(planBuilder, pinotTable, regionId, city, fare, secondsSinceEpoch), getRowExpression("secondssinceepoch > 20", defaultSessionHolder)), ImmutableList.of("city", "secondssinceepoch"))), + "SELECT city, secondsSinceEpoch FROM realtimeOnly WHERE (secondsSinceEpoch > 20) LIMIT 50"); + } + + @Test + public void testCountStar() + { + testUnaryAggregationHelper((planBuilder, aggregationBuilder) -> aggregationBuilder.addAggregation(planBuilder.variable("agg"), getRowExpression("count(*)", defaultSessionHolder)), "count(*)"); + } + + @Test + public void testDistinctSelection() + { + PlanNode justScan = buildPlan(planBuilder -> tableScan(planBuilder, pinotTable, regionId, secondsSinceEpoch, city, fare)); + testPQL(planBuilder -> planBuilder.aggregation(aggBuilder -> aggBuilder.source(justScan).singleGroupingSet(v("regionid"))), + "SELECT count(*) FROM realtimeOnly GROUP BY regionId TOP 10000"); + } + + @Test + public void testPercentileAggregation() + { + testUnaryAggregationHelper((planBuilder, aggregationBuilder) -> aggregationBuilder.addAggregation(planBuilder.variable("agg"), getRowExpression("approx_percentile(fare, 0.10)", defaultSessionHolder)), "PERCENTILEEST10(fare)"); + } + + @Test + public void testApproxDistinct() + { + testUnaryAggregationHelper((planBuilder, aggregationBuilder) -> aggregationBuilder.addAggregation(planBuilder.variable("agg"), getRowExpression("approx_distinct(fare)", defaultSessionHolder)), "DISTINCTCOUNTHLL(fare)"); + } + + @Test + public void testAggWithUDFInGroupBy() + { + LinkedHashMap aggProjection = new LinkedHashMap<>(); + aggProjection.put("date", "date_trunc('day', cast(from_unixtime(secondssinceepoch - 50) AS TIMESTAMP))"); + PlanNode justDate = buildPlan(planBuilder -> project(planBuilder, tableScan(planBuilder, pinotTable, regionId, secondsSinceEpoch, city, fare), aggProjection, defaultSessionHolder)); + testPQL(planBuilder -> planBuilder.aggregation(aggBuilder -> aggBuilder.source(justDate).singleGroupingSet(new VariableReferenceExpression("date", TIMESTAMP)).addAggregation(planBuilder.variable("agg"), getRowExpression("count(*)", defaultSessionHolder))), + "SELECT count(*) FROM realtimeOnly GROUP BY dateTimeConvert(SUB(secondsSinceEpoch, 50), '1:SECONDS:EPOCH', '1:MILLISECONDS:EPOCH', '1:DAYS') TOP 10000"); + aggProjection.put("city", "city"); + PlanNode newScanWithCity = buildPlan(planBuilder -> project(planBuilder, tableScan(planBuilder, pinotTable, regionId, secondsSinceEpoch, city, fare), aggProjection, defaultSessionHolder)); + testPQL(planBuilder -> planBuilder.aggregation(aggBuilder -> aggBuilder.source(newScanWithCity).singleGroupingSet(new VariableReferenceExpression("date", TIMESTAMP), v("city")).addAggregation(planBuilder.variable("agg"), getRowExpression("count(*)", defaultSessionHolder))), + "SELECT count(*) FROM realtimeOnly GROUP BY dateTimeConvert(SUB(secondsSinceEpoch, 50), '1:SECONDS:EPOCH', '1:MILLISECONDS:EPOCH', '1:DAYS'), city TOP 10000"); + } + + @Test + public void testMultipleAggregatesWithOutGroupBy() + { + Map outputVariables = ImmutableMap.of("agg", "count(*)", "min", "min(fare)"); + PlanNode justScan = buildPlan(planBuilder -> tableScan(planBuilder, pinotTable, regionId, secondsSinceEpoch, city, fare)); + testPQL(planBuilder -> planBuilder.aggregation(aggBuilder -> aggBuilder.source(justScan).globalGrouping().addAggregation(planBuilder.variable("agg"), getRowExpression("count(*)", defaultSessionHolder)).addAggregation(planBuilder.variable("min"), getRowExpression("min(fare)", defaultSessionHolder))), + "SELECT __expressions__ FROM realtimeOnly", defaultSessionHolder, outputVariables); + testPQL(planBuilder -> planBuilder.limit(50L, planBuilder.aggregation(aggBuilder -> aggBuilder.source(justScan).globalGrouping().addAggregation(planBuilder.variable("agg"), getRowExpression("count(*)", defaultSessionHolder)).addAggregation(planBuilder.variable("min"), getRowExpression("min(fare)", defaultSessionHolder)))), + "SELECT __expressions__ FROM realtimeOnly", defaultSessionHolder, outputVariables); + } + + @Test + public void testMultipleAggregatesWhenAllowed() + { + helperTestMultipleAggregatesWithGroupBy(new PinotConfig().setAllowMultipleAggregations(true)); + } + + @Test(expectedExceptions = NoSuchElementException.class) + public void testMultipleAggregatesNotAllowed() + { + helperTestMultipleAggregatesWithGroupBy(pinotConfig); + } + + private void helperTestMultipleAggregatesWithGroupBy(PinotConfig givenPinotConfig) + { + Map outputVariables = ImmutableMap.of("agg", "count(*)", "min", "min(fare)"); + PlanNode justScan = buildPlan(planBuilder -> tableScan(planBuilder, pinotTable, regionId, secondsSinceEpoch, city, fare)); + testPQL(givenPinotConfig, planBuilder -> planBuilder.aggregation(aggBuilder -> aggBuilder.source(justScan).singleGroupingSet(v("city")).addAggregation(planBuilder.variable("agg"), getRowExpression("count(*)", defaultSessionHolder)).addAggregation(planBuilder.variable("min"), getRowExpression("min(fare)", defaultSessionHolder))), + "SELECT __expressions__ FROM realtimeOnly GROUP BY city TOP 10000", defaultSessionHolder, outputVariables); + } + + @Test(expectedExceptions = NoSuchElementException.class) + public void testMultipleAggregateGroupByWithLimitFails() + { + Map outputVariables = ImmutableMap.of("agg", "count(*)", "min", "min(fare)"); + PlanNode justScan = buildPlan(planBuilder -> tableScan(planBuilder, pinotTable, regionId, secondsSinceEpoch, city, fare)); + testPQL(planBuilder -> planBuilder.limit(50L, planBuilder.aggregation(aggBuilder -> aggBuilder.source(justScan).singleGroupingSet(v("city")).addAggregation(planBuilder.variable("agg"), getRowExpression("count(*)", defaultSessionHolder)).addAggregation(planBuilder.variable("min"), getRowExpression("min(fare)", defaultSessionHolder)))), + "SELECT __expressions__ FROM realtimeOnly GROUP BY city TOP 50", defaultSessionHolder, outputVariables); + } + + @Test(expectedExceptions = NoSuchElementException.class) + public void testForbiddenProjectionOutsideOfAggregation() + { + LinkedHashMap projections = new LinkedHashMap<>(ImmutableMap.of("hour", "date_trunc('hour', from_unixtime(secondssinceepoch))", "regionid", "regionid")); + PlanNode plan = buildPlan(planBuilder -> limit(planBuilder, 10, project(planBuilder, tableScan(planBuilder, pinotTable, secondsSinceEpoch, regionId), projections, defaultSessionHolder))); + testPQL(pinotConfig, plan, "Should fail", defaultSessionHolder, ImmutableMap.of()); + } + + @Test + public void testSimpleSelectWithTopN() + { + PlanBuilder planBuilder = createPlanBuilder(defaultSessionHolder); + TableScanNode tableScanNode = tableScan(planBuilder, pinotTable, regionId, city, fare); + TopNNode topNFare = topN(planBuilder, 50L, ImmutableList.of("fare"), ImmutableList.of(false), tableScanNode); + testPQL(pinotConfig, topNFare, + "SELECT regionId, city, fare FROM realtimeOnly ORDER BY fare DESC LIMIT 50", defaultSessionHolder, ImmutableMap.of()); + TopNNode topnFareAndCity = topN(planBuilder, 50L, ImmutableList.of("fare", "city"), ImmutableList.of(true, false), tableScanNode); + testPQL(pinotConfig, topnFareAndCity, + "SELECT regionId, city, fare FROM realtimeOnly ORDER BY fare, city DESC LIMIT 50", defaultSessionHolder, ImmutableMap.of()); + } + + @Test(expectedExceptions = NoSuchElementException.class) + public void testAggregationWithOrderByPushDownInTopN() + { + PlanBuilder planBuilder = createPlanBuilder(defaultSessionHolder); + TableScanNode tableScanNode = tableScan(planBuilder, pinotTable, city, fare); + AggregationNode agg = planBuilder.aggregation(aggBuilder -> aggBuilder.source(tableScanNode).singleGroupingSet(v("city")).addAggregation(planBuilder.variable("agg"), getRowExpression("sum(fare)", defaultSessionHolder))); + TopNNode topN = new TopNNode(planBuilder.getIdAllocator().getNextId(), agg, 50L, new OrderingScheme(ImmutableList.of(new Ordering(v("city"), SortOrder.DESC_NULLS_FIRST))), TopNNode.Step.FINAL); + testPQL(pinotConfig, topN, "", defaultSessionHolder, ImmutableMap.of()); + } +} diff --git a/presto-pinot/pom.xml b/presto-pinot/pom.xml new file mode 100644 index 0000000000000..99f42439cee55 --- /dev/null +++ b/presto-pinot/pom.xml @@ -0,0 +1,64 @@ + + + 4.0.0 + + com.facebook.presto + presto-root + 0.229-SNAPSHOT + + + presto-pinot + presto-pinot + Presto - Pinot Native Connector + presto-plugin + + + ${project.parent.basedir} + + + + + com.facebook.presto + presto-pinot-toolkit + ${project.version} + + + + com.google.guava + guava + + + + + com.facebook.presto + presto-spi + provided + + + + org.openjdk.jol + jol-core + provided + + + + io.airlift + slice + provided + + + + io.airlift + units + provided + + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotPlugin.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotPlugin.java new file mode 100644 index 0000000000000..7f18c769a5398 --- /dev/null +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotPlugin.java @@ -0,0 +1,28 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.pinot; + +import com.facebook.presto.spi.Plugin; +import com.facebook.presto.spi.connector.ConnectorFactory; +import com.google.common.collect.ImmutableList; + +public class PinotPlugin + implements Plugin +{ + @Override + public Iterable getConnectorFactories() + { + return ImmutableList.of(new PinotConnectorFactory()); + } +} diff --git a/presto-pinot/src/main/resources/META-INF/services/com.facebook.presto.spi.Plugin b/presto-pinot/src/main/resources/META-INF/services/com.facebook.presto.spi.Plugin new file mode 100644 index 0000000000000..5d236ddb0c2a4 --- /dev/null +++ b/presto-pinot/src/main/resources/META-INF/services/com.facebook.presto.spi.Plugin @@ -0,0 +1,2 @@ +com.facebook.presto.pinot.PinotPlugin + diff --git a/presto-server/pom.xml b/presto-server/pom.xml index a06254e9fde81..249bc86d41296 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -92,6 +92,14 @@ provided + + com.facebook.presto + presto-pinot + ${project.version} + zip + provided + + com.facebook.presto presto-example-http diff --git a/presto-server/src/main/assembly/presto.xml b/presto-server/src/main/assembly/presto.xml index efd14285cc6c8..7df731e7e30e6 100644 --- a/presto-server/src/main/assembly/presto.xml +++ b/presto-server/src/main/assembly/presto.xml @@ -72,6 +72,10 @@ ${project.build.directory}/dependency/presto-cassandra-${project.version} plugin/cassandra + + ${project.build.directory}/dependency/presto-pinot-${project.version} + plugin/pinot + ${project.build.directory}/dependency/presto-example-http-${project.version} plugin/example-http diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/ConnectorPlanOptimizer.java b/presto-spi/src/main/java/com/facebook/presto/spi/ConnectorPlanOptimizer.java index 6d24cbdf18a1d..1915138d33c03 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/ConnectorPlanOptimizer.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/ConnectorPlanOptimizer.java @@ -13,7 +13,6 @@ */ package com.facebook.presto.spi; -import com.facebook.presto.spi.api.Experimental; import com.facebook.presto.spi.plan.PlanNode; import com.facebook.presto.spi.plan.PlanNodeIdAllocator; @@ -27,10 +26,7 @@ * There could be multiple PlanNodes satisfying the above conditions. * All of them will be processed with the given implementation of ConnectorPlanOptimizer. * Each optimization is processed exactly once at the end of logical planning (i.e. right before AddExchanges). - * TODO: currently, optimization is processed at the end of physical planning given only filter and table scan nodes are supported. - * TODO: the connector optimization will be moved to the end of logical planning after #12828 */ -@Experimental public interface ConnectorPlanOptimizer { PlanNode optimize( diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/connector/ConnectorPlanOptimizerProvider.java b/presto-spi/src/main/java/com/facebook/presto/spi/connector/ConnectorPlanOptimizerProvider.java index 08b755e350405..c05b248c87a64 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/connector/ConnectorPlanOptimizerProvider.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/connector/ConnectorPlanOptimizerProvider.java @@ -19,5 +19,14 @@ public interface ConnectorPlanOptimizerProvider { - Set getConnectorPlanOptimizers(); + /** + * The plan optimizers to be applied before having the notion of distribution. + */ + Set getLogicalPlanOptimizers(); + + /** + * The plan optimizers to be applied after having the notion of distribution. + * The plan will be only executed on a single node. + */ + Set getPhysicalPlanOptimizers(); }