From 25b39bcc6320de63905e97ec5a49974def59c6df Mon Sep 17 00:00:00 2001 From: Haibo Wang Date: Thu, 12 Sep 2019 22:26:22 -0700 Subject: [PATCH 1/2] Add Pinot connector --- pom.xml | 10 + presto-main/etc/catalog/pinot.properties | 4 + presto-main/etc/config.properties | 1 + presto-pinot/pom.xml | 434 ++++++++++++++++++ .../presto/pinot/PinotClusterInfoFetcher.java | 201 ++++++++ .../facebook/presto/pinot/PinotColumn.java | 78 ++++ .../presto/pinot/PinotColumnHandle.java | 109 +++++ .../presto/pinot/PinotColumnMetadata.java | 107 +++++ .../presto/pinot/PinotColumnUtils.java | 74 +++ .../facebook/presto/pinot/PinotConfig.java | 260 +++++++++++ .../presto/pinot/PinotConnection.java | 100 ++++ .../facebook/presto/pinot/PinotConnector.java | 83 ++++ .../presto/pinot/PinotConnectorFactory.java | 67 +++ .../presto/pinot/PinotConnectorId.java | 54 +++ .../facebook/presto/pinot/PinotErrorCode.java | 49 ++ .../presto/pinot/PinotHandleResolver.java | 55 +++ .../facebook/presto/pinot/PinotMetadata.java | 234 ++++++++++ .../facebook/presto/pinot/PinotModule.java | 77 ++++ .../presto/pinot/PinotPageSource.java | 399 ++++++++++++++++ .../presto/pinot/PinotPageSourceProvider.java | 67 +++ .../facebook/presto/pinot/PinotPlugin.java | 28 ++ .../presto/pinot/PinotQueryGenerator.java | 80 ++++ .../pinot/PinotScatterGatherQueryClient.java | 319 +++++++++++++ .../com/facebook/presto/pinot/PinotSplit.java | 131 ++++++ .../presto/pinot/PinotSplitManager.java | 287 ++++++++++++ .../com/facebook/presto/pinot/PinotTable.java | 65 +++ .../presto/pinot/PinotTableHandle.java | 105 +++++ .../presto/pinot/PinotTableLayoutHandle.java | 64 +++ .../presto/pinot/PinotTransactionHandle.java | 22 + .../java/com/facebook/presto/pinot/Types.java | 33 ++ .../services/com.facebook.presto.spi.Plugin | 1 + .../facebook/presto/pinot/MetadataUtil.java | 76 +++ .../presto/pinot/PinotQueryGeneratorTest.java | 57 +++ .../presto/pinot/PinotSplitManagerTest.java | 178 +++++++ .../presto/pinot/TestPinotColumnHandle.java | 58 +++ .../presto/pinot/TestPinotConfig.java | 76 +++ .../presto/pinot/TestPinotTableHandle.java | 41 ++ presto-server/src/main/assembly/presto.xml | 4 + 38 files changed, 4088 insertions(+) create mode 100644 presto-main/etc/catalog/pinot.properties create mode 100644 presto-pinot/pom.xml create mode 100644 presto-pinot/src/main/java/com/facebook/presto/pinot/PinotClusterInfoFetcher.java create mode 100644 presto-pinot/src/main/java/com/facebook/presto/pinot/PinotColumn.java create mode 100644 presto-pinot/src/main/java/com/facebook/presto/pinot/PinotColumnHandle.java create mode 100644 presto-pinot/src/main/java/com/facebook/presto/pinot/PinotColumnMetadata.java create mode 100644 presto-pinot/src/main/java/com/facebook/presto/pinot/PinotColumnUtils.java create mode 100644 presto-pinot/src/main/java/com/facebook/presto/pinot/PinotConfig.java create mode 100644 presto-pinot/src/main/java/com/facebook/presto/pinot/PinotConnection.java create mode 100644 presto-pinot/src/main/java/com/facebook/presto/pinot/PinotConnector.java create mode 100644 presto-pinot/src/main/java/com/facebook/presto/pinot/PinotConnectorFactory.java create mode 100644 presto-pinot/src/main/java/com/facebook/presto/pinot/PinotConnectorId.java create mode 100644 presto-pinot/src/main/java/com/facebook/presto/pinot/PinotErrorCode.java create mode 100644 presto-pinot/src/main/java/com/facebook/presto/pinot/PinotHandleResolver.java create mode 100644 presto-pinot/src/main/java/com/facebook/presto/pinot/PinotMetadata.java create mode 100644 presto-pinot/src/main/java/com/facebook/presto/pinot/PinotModule.java create mode 100644 presto-pinot/src/main/java/com/facebook/presto/pinot/PinotPageSource.java create mode 100644 presto-pinot/src/main/java/com/facebook/presto/pinot/PinotPageSourceProvider.java create mode 100644 presto-pinot/src/main/java/com/facebook/presto/pinot/PinotPlugin.java create mode 100644 presto-pinot/src/main/java/com/facebook/presto/pinot/PinotQueryGenerator.java create mode 100644 presto-pinot/src/main/java/com/facebook/presto/pinot/PinotScatterGatherQueryClient.java create mode 100644 presto-pinot/src/main/java/com/facebook/presto/pinot/PinotSplit.java create mode 100644 presto-pinot/src/main/java/com/facebook/presto/pinot/PinotSplitManager.java create mode 100644 presto-pinot/src/main/java/com/facebook/presto/pinot/PinotTable.java create mode 100644 presto-pinot/src/main/java/com/facebook/presto/pinot/PinotTableHandle.java create mode 100644 presto-pinot/src/main/java/com/facebook/presto/pinot/PinotTableLayoutHandle.java create mode 100644 presto-pinot/src/main/java/com/facebook/presto/pinot/PinotTransactionHandle.java create mode 100644 presto-pinot/src/main/java/com/facebook/presto/pinot/Types.java create mode 100644 presto-pinot/src/main/resources/META-INF/services/com.facebook.presto.spi.Plugin create mode 100644 presto-pinot/src/test/java/com/facebook/presto/pinot/MetadataUtil.java create mode 100644 presto-pinot/src/test/java/com/facebook/presto/pinot/PinotQueryGeneratorTest.java create mode 100644 presto-pinot/src/test/java/com/facebook/presto/pinot/PinotSplitManagerTest.java create mode 100644 presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotColumnHandle.java create mode 100644 presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotConfig.java create mode 100644 presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotTableHandle.java diff --git a/pom.xml b/pom.xml index c7aeb9dc3322e..526d41ba2af42 100644 --- a/pom.xml +++ b/pom.xml @@ -61,6 +61,15 @@ 1.6.8 6.2.1 1.9.17 + 4.4.6 + 4.5.3 + 2.2.0 + 1.1.24 + 3.10.6.Final + 4.1.28.Final + 2.0.1.Final + 2.28 + 0.1.0 + + org.apache.pinot + pinot-api + ${dep.pinot.version} + + + log4j + log4j + + + com.ning + async-http-client + + + org.slf4j + slf4j-log4j12 + + + org.slf4j + slf4j-api + + + com.101tec + zkclient + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-databind + + + + + + org.apache.pinot + pinot-common + ${dep.pinot.version} + + + log4j + log4j + + + org.slf4j + slf4j-api + + + org.slf4j + slf4j-log4j12 + + + commons-logging + commons-logging + + + org.antlr + antlr4-annotations + + + org.apache.kafka + kafka-clients + + + org.codehaus.jackson + jackson-mapper-asl + + + org.apache.kafka + kafka_2.10 + + + javax.servlet + javax.servlet-api + + + org.glassfish.hk2.external + jakarta.inject + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-databind + + + commons-beanutils + commons-beanutils-core + + + jakarta.annotation + jakarta.annotation-api + + + jakarta.ws.rs + jakarta.ws.rs-api + + + org.apache.avro + avro + + + + + + org.apache.pinot + pinot-transport + ${dep.pinot.version} + + + log4j + log4j + + + org.slf4j + slf4j-api + + + org.slf4j + slf4j-log4j12 + + + org.antlr + antlr4-annotations + + + org.apache.kafka + kafka-clients + + + org.apache.kafka + kafka_2.10 + + + org.glassfish.jersey.core + jersey-server + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-databind + + + commons-beanutils + commons-beanutils-core + + + io.netty + netty-all + + + + + + org.apache.pinot + pinot-core + ${dep.pinot.version} + + + log4j + log4j + + + org.slf4j + slf4j-api + + + org.slf4j + slf4j-log4j12 + + + org.antlr + antlr4-annotations + + + it.unimi.dsi + fastutil + + + org.apache.kafka + kafka-clients + + + org.apache.kafka + kafka_2.10 + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-databind + + + commons-beanutils + commons-beanutils-core + + + org.codehaus.jackson + jackson-core-asl + + + io.netty + netty-all + + + + + + + + io.airlift + bootstrap + + + org.slf4j + jcl-over-slf4j + + + + + + io.airlift + json + + + + io.airlift + log + + + + io.airlift + configuration + + + + io.airlift + units + provided + + + + com.google.guava + guava + + + + com.google.inject + guice + + + + javax.validation + validation-api + ${dep.validation-api.version} + + + + 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 + + + + + com.facebook.presto + presto-main + test + + + + org.testng + testng + test + + + + io.airlift + testing + test + + + + io.airlift + http-server + test + + + + io.airlift + node + test + + + + javax.servlet + javax.servlet-api + test + + + org.apache.thrift + libthrift + + + com.yammer.metrics + metrics-core + ${dep.yammer.version} + + + org.apache.httpcomponents + httpcore + ${dep.httpcore.version} + + + commons-logging + commons-logging + + + + + + org.apache.httpcomponents + httpclient + ${dep.httpclient.version} + + + diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotClusterInfoFetcher.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotClusterInfoFetcher.java new file mode 100644 index 0000000000000..c29a724abe2ba --- /dev/null +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotClusterInfoFetcher.java @@ -0,0 +1,201 @@ +/* + * 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.alibaba.fastjson.JSONArray; +import com.alibaba.fastjson.JSONObject; +import com.facebook.presto.spi.PrestoException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; +import io.airlift.http.client.HttpStatus; +import io.airlift.log.Logger; +import org.apache.http.HttpEntity; +import org.apache.http.HttpHeaders; +import org.apache.http.HttpResponse; +import org.apache.http.client.ClientProtocolException; +import org.apache.http.client.ResponseHandler; +import org.apache.http.client.methods.HttpUriRequest; +import org.apache.http.client.methods.RequestBuilder; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClients; +import org.apache.http.util.EntityUtils; +import org.apache.pinot.client.DynamicBrokerSelector; +import org.apache.pinot.common.data.Schema; +import org.apache.pinot.common.utils.NetUtil; + +import java.io.IOException; +import java.net.SocketException; +import java.net.UnknownHostException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; + +import static com.facebook.presto.pinot.PinotErrorCode.PINOT_FAILURE_GETTING_TABLE; +import static io.airlift.http.client.HttpStatus.familyForStatusCode; +import static org.apache.pinot.common.config.TableNameBuilder.extractRawTableName; + +/** + * This class handles fetching necessary information from the Pinot cluster, including table schema, broker host, routing table, etc + */ +public class PinotClusterInfoFetcher +{ + private static final String APPLICATION_JSON = "application/json"; + + private static final String GET_ALL_TABLES_API_TEMPLATE = "http://%s/tables"; + private static final String TABLE_SCHEMA_API_TEMPLATE = "http://%s/tables/%s/schema"; + private static final String ROUTING_TABLE_API_TEMPLATE = "http://%s/debug/routingTable/%s"; + private static final String TIME_BOUNDARY_API_TEMPLATE = "http://%s/debug/timeBoundary/%s"; + + private static final Logger log = Logger.get(PinotClusterInfoFetcher.class); + + private static final CloseableHttpClient HTTP_CLIENT = HttpClients.createDefault(); + private final String controllerUrl; + private final String zookeeperServers; + private DynamicBrokerSelector dynamicBrokerSelector; + private String instanceId = "Presto_pinot_master"; + private ObjectMapper objectMapper = new ObjectMapper(); + + @Inject + public PinotClusterInfoFetcher(PinotConfig pinotConfig) + throws SocketException, UnknownHostException + { + this(pinotConfig.getZookeeperUrl(), pinotConfig.getPinotCluster(), pinotConfig.getControllerUrl()); + } + + public PinotClusterInfoFetcher(String zookeeperUrl, String pinotCluster, String controllerUrl) + throws SocketException, UnknownHostException + { + log.info("Trying to init PinotClusterInfoFetcher with Zookeeper: %s, PinotCluster %s, ControllerUrl: %s.", zookeeperUrl, pinotCluster, controllerUrl); + zookeeperServers = zookeeperUrl + "/" + pinotCluster; + instanceId = instanceId + "_" + NetUtil.getHostAddress(); + this.controllerUrl = controllerUrl; + } + + public static String sendHttpGet(final String url) + throws Exception + { + HttpUriRequest request = RequestBuilder.get(url).setHeader(HttpHeaders.CONTENT_TYPE, APPLICATION_JSON).build(); + return HTTP_CLIENT.execute(request, getStringResponseHandler()); + } + + private static ResponseHandler getStringResponseHandler() + { + return new ResponseHandler() + { + @Override + public String handleResponse(HttpResponse response) + throws ClientProtocolException, IOException + { + int status = response.getStatusLine().getStatusCode(); + HttpEntity entity = response.getEntity(); + if (familyForStatusCode(response.getStatusLine().getStatusCode()) == HttpStatus.Family.SUCCESSFUL && entity != null) { + return EntityUtils.toString(entity); + } + else { + if (entity == null) { + throw new ClientProtocolException("Entity is null"); + } + throw new ClientProtocolException("Unexpected status: " + status); + } + } + }; + } + + public void close() + throws IOException + { + HTTP_CLIENT.close(); + } + + private String getControllerUrl() + { + return this.controllerUrl; + } + + @SuppressWarnings("unchecked") + public List getAllTables() + throws Exception + { + final String url = String.format(GET_ALL_TABLES_API_TEMPLATE, getControllerUrl()); + String responseBody = sendHttpGet(url); + Map> responseMap = objectMapper.readValue(responseBody, Map.class); + return responseMap.get("tables"); + } + + public Schema getTableSchema(String table) + throws Exception + { + final String url = String.format(TABLE_SCHEMA_API_TEMPLATE, getControllerUrl(), table); + String responseBody = sendHttpGet(url); + return Schema.fromString(responseBody); + } + + public String getBrokerHost(String table) + throws Exception + { + if (dynamicBrokerSelector == null) { + dynamicBrokerSelector = new DynamicBrokerSelector(zookeeperServers); + } + return this.dynamicBrokerSelector.selectBroker(table); + } + + public Map>> getRoutingTableForTable(String tableName) + throws Exception + { + final Map>> routingTableMap = new HashMap<>(); + final String url = String.format(ROUTING_TABLE_API_TEMPLATE, getBrokerHost(tableName), tableName); + String responseBody = sendHttpGet(url); + log.debug("Trying to get routingTable for %s. url: %s", tableName, url); + JSONObject resp = JSONObject.parseObject(responseBody); + JSONArray routingTableSnapshots = resp.getJSONArray("routingTableSnapshot"); + for (int i = 0; i < routingTableSnapshots.size(); i++) { + JSONObject snapshot = routingTableSnapshots.getJSONObject(i); + String tableNameWithType = snapshot.getString("tableName"); + // 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". + // The bug is fixed in the latest Pinot but not in the release version 0.1.0 + if (!tableName.equals(extractRawTableName(tableNameWithType))) { + log.debug("Ignoring routingTable for %s", tableNameWithType); + continue; + } + JSONArray routingTableEntriesArray = snapshot.getJSONArray("routingTableEntries"); + if (routingTableEntriesArray.size() == 0) { + throw new PrestoException( + PINOT_FAILURE_GETTING_TABLE, + "RoutingTable is empty for " + tableName); + } + String routingTableEntries = routingTableEntriesArray.getJSONObject(new Random().nextInt(routingTableEntriesArray.size())).toJSONString(); + Map> routingTable = objectMapper.readValue(routingTableEntries, Map.class); + routingTableMap.put(tableNameWithType, routingTable); + } + return routingTableMap; + } + + public Map getTimeBoundaryForTable(String table) + throws Exception + { + final String url = String.format(TIME_BOUNDARY_API_TEMPLATE, getBrokerHost(table), table); + String responseBody = sendHttpGet(url); + JSONObject resp = JSONObject.parseObject(responseBody); + Map timeBoundary = new HashMap<>(); + if (resp.containsKey("timeColumnName")) { + timeBoundary.put("timeColumnName", resp.getString("timeColumnName")); + } + if (resp.containsKey("timeColumnValue")) { + timeBoundary.put("timeColumnValue", resp.getString("timeColumnValue")); + } + return timeBoundary; + } +} diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotColumn.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotColumn.java new file mode 100644 index 0000000000000..8c4f0a1af45f6 --- /dev/null +++ b/presto-pinot/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/src/main/java/com/facebook/presto/pinot/PinotColumnHandle.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotColumnHandle.java new file mode 100644 index 0000000000000..4e6e4b8117942 --- /dev/null +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotColumnHandle.java @@ -0,0 +1,109 @@ +/* + * 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.type.Type; +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 PinotColumnHandle + implements ColumnHandle +{ + private final String connectorId; + private final String columnName; + private final Type columnType; + private final int ordinalPosition; + private Optional aggregationType; + + @JsonCreator + public PinotColumnHandle( + @JsonProperty("connectorId") String connectorId, + @JsonProperty("columnName") String columnName, + @JsonProperty("columnType") Type columnType, + @JsonProperty("ordinalPosition") int ordinalPosition) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null"); + this.columnName = requireNonNull(columnName, "columnName is null"); + this.columnType = requireNonNull(columnType, "columnType is null"); + this.ordinalPosition = ordinalPosition; + } + + @JsonProperty + public String getConnectorId() + { + return connectorId; + } + + @JsonProperty + public String getColumnName() + { + return columnName; + } + + @JsonProperty + public Type getColumnType() + { + return columnType; + } + + @JsonProperty + public int getOrdinalPosition() + { + return ordinalPosition; + } + + public ColumnMetadata getColumnMetadata() + { + return new ColumnMetadata(columnName, columnType); + } + + public String getName() + { + return columnName; + } + + @Override + public int hashCode() + { + return Objects.hash(connectorId, columnName); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if ((obj == null) || (getClass() != obj.getClass())) { + return false; + } + + PinotColumnHandle other = (PinotColumnHandle) obj; + return Objects.equals(this.connectorId, other.connectorId) && Objects.equals(this.columnName, other.columnName); + } + + @Override + public String toString() + { + return toStringHelper(this).add("connectorId", connectorId).add("columnName", columnName).add("columnType", columnType).add("ordinalPosition", ordinalPosition).toString(); + } +} diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotColumnMetadata.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotColumnMetadata.java new file mode 100644 index 0000000000000..6ad1494df5fab --- /dev/null +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotColumnMetadata.java @@ -0,0 +1,107 @@ +/* + * 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 com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Locale.ENGLISH; + +public class PinotColumnMetadata + extends ColumnMetadata +{ + private final String name; + private final Type type; + private final String comment; + private final boolean isHidden; + + public PinotColumnMetadata(String name, Type type) + { + this(name, type, null, false); + } + + public PinotColumnMetadata(String name, Type type, String comment, boolean isHidden) + { + super(name, type, comment, isHidden); + if (name == null || name.isEmpty()) { + throw new NullPointerException("name is null or empty"); + } + if (type == null) { + throw new NullPointerException("type is null"); + } + + this.name = name; + this.type = type; + this.comment = comment; + this.isHidden = isHidden; + } + + public String getName() + { + return name.toLowerCase(ENGLISH); + } + + public String getPinotName() + { + return name; + } + + public Type getType() + { + return type; + } + + public String getComment() + { + return comment; + } + + public boolean isHidden() + { + return isHidden; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("name", name) + .add("type", type) + .add("comment", comment) + .add("isHidden", isHidden) + .toString(); + } + + @Override + public int hashCode() + { + return Objects.hash(name, type, comment, 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.type, other.type) && Objects.equals(this.comment, other.comment) && Objects.equals(this.isHidden, other.isHidden); + } +} diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotColumnUtils.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotColumnUtils.java new file mode 100644 index 0000000000000..746cecce6e855 --- /dev/null +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotColumnUtils.java @@ -0,0 +1,74 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package 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.ArrayList; +import java.util.List; + +public class PinotColumnUtils +{ + private PinotColumnUtils() + { + } + + public static List getPinotColumnsForPinotSchema(Schema pinotTableSchema) + { + List pinotColumns = new ArrayList<>(); + for (String columnName : pinotTableSchema.getColumnNames()) { + PinotColumn pinotColumn = new PinotColumn(columnName, getPrestoTypeFromPinotType(pinotTableSchema.getFieldSpecFor(columnName))); + pinotColumns.add(pinotColumn); + } + return pinotColumns; + } + + public static Type getPrestoTypeFromPinotType(FieldSpec pinotFieldSpecification) + { + if (pinotFieldSpecification.isSingleValueField()) { + return getPrestoTypeFromPinotType(pinotFieldSpecification.getDataType()); + } + return VarcharType.VARCHAR; + } + + public static Type getPrestoTypeFromPinotType(DataType dataType) + { + switch (dataType) { + case BOOLEAN: + return BooleanType.BOOLEAN; + case BYTES: + return VarcharType.createVarcharType(1); + 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 UnsupportedOperationException("Not support type conversion for pinot data type: " + dataType); + } +} diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotConfig.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotConfig.java new file mode 100644 index 0000000000000..4a3cfcb7664d0 --- /dev/null +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotConfig.java @@ -0,0 +1,260 @@ +/* + * 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 io.airlift.configuration.Config; +import io.airlift.units.Duration; +import io.airlift.units.MinDuration; + +import javax.validation.constraints.NotNull; + +import java.util.concurrent.TimeUnit; + +public class PinotConfig +{ + 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_MIN_CONNECTIONS_PER_SERVER = 10; + private static final int DEFAULT_MAX_CONNECTIONS_PER_SERVER = 30; + private static final int DEFAULT_MAX_BACKLOG_PER_SERVER = 30; + private static final int DEFAULT_THREAD_POOL_SIZE = 64; + private static final int DEFAULT_CORE_POOL_SIZE = 50; + + private static final long DEFAULT_LIMIT_ALL = 2147483647; + + private static final int DEFAULT_ESTIMATED_SIZE_IN_BYTES_FOR_NON_NUMERIC_COLUMN = 20; + + private String zookeeperUrl; + private String pinotCluster; + private String controllerUrl; + + private long limitAll = DEFAULT_LIMIT_ALL; + + private Duration idleTimeout = DEFAULT_IDLE_TIMEOUT; + private Duration connectionTimeout = DEFAULT_CONNECTION_TIMEOUT; + + // Maximum number of threads actively processing tasks. + private int threadPoolSize = DEFAULT_THREAD_POOL_SIZE; + + // The number of threads to keep in the pool when connecting with Pinot, even if they are idle + private int corePoolSize = DEFAULT_CORE_POOL_SIZE; + + // Minimum number of live connections for each server + private int minConnectionsPerServer = DEFAULT_MIN_CONNECTIONS_PER_SERVER; + + // Maximum number of live connections for each server + private int maxConnectionsPerServer = DEFAULT_MAX_CONNECTIONS_PER_SERVER; + + // Maximum number of pending checkout requests before requests starts getting rejected + private int maxBacklogPerServer = DEFAULT_MAX_BACKLOG_PER_SERVER; + + // Estimated size for non-numeric columns, used when processing Pages. Use heuristics to save calculating actual bytes at query time. + private int estimatedSizeInBytesForNonNumericColumn = DEFAULT_ESTIMATED_SIZE_IN_BYTES_FOR_NON_NUMERIC_COLUMN; + + @NotNull + public String getZookeeperUrl() + { + return zookeeperUrl; + } + + @Config("zookeeper-uri") + public PinotConfig setZookeeperUrl(String zookeeperUrl) + { + if (zookeeperUrl != null && zookeeperUrl.endsWith("/")) { + zookeeperUrl = zookeeperUrl.substring(0, zookeeperUrl.length() - 1); + } + this.zookeeperUrl = zookeeperUrl; + return this; + } + + @NotNull + public String getPinotCluster() + { + return pinotCluster; + } + + @Config("pinot-cluster") + public PinotConfig setPinotCluster(String pinotCluster) + { + this.pinotCluster = pinotCluster; + return this; + } + + @NotNull + public String getControllerUrl() + { + return controllerUrl; + } + + @Config("controller-url") + public PinotConfig setControllerUrl(String controllerUrl) + { + this.controllerUrl = controllerUrl; + return this; + } + + @NotNull + public long getLimitAll() + { + return limitAll; + } + + @Config("limit-all") + public PinotConfig setLimitAll(String limitAll) + { + try { + this.limitAll = Long.valueOf(limitAll); + } + catch (Exception e) { + this.limitAll = DEFAULT_LIMIT_ALL; + } + return this; + } + + @NotNull + public int getThreadPoolSize() + { + return threadPoolSize; + } + + @Config("thread-pool-size") + public PinotConfig setThreadPoolSize(String threadPoolSize) + { + try { + this.threadPoolSize = Integer.valueOf(threadPoolSize); + } + catch (Exception e) { + this.threadPoolSize = DEFAULT_THREAD_POOL_SIZE; + } + return this; + } + + @NotNull + public int getCorePoolSize() + { + return corePoolSize; + } + + @Config("core-pool-size") + public PinotConfig setCorePoolSize(String corePoolSize) + { + try { + this.corePoolSize = Integer.valueOf(corePoolSize); + } + catch (Exception e) { + this.corePoolSize = DEFAULT_CORE_POOL_SIZE; + } + return this; + } + + @NotNull + public int getMinConnectionsPerServer() + { + return minConnectionsPerServer; + } + + @Config("min-connections-per-server") + public PinotConfig setMinConnectionsPerServer(String minConnectionsPerServer) + { + try { + this.minConnectionsPerServer = Integer.valueOf(minConnectionsPerServer); + } + catch (Exception e) { + this.minConnectionsPerServer = DEFAULT_MIN_CONNECTIONS_PER_SERVER; + } + return this; + } + + @NotNull + public int getMaxConnectionsPerServer() + { + return maxConnectionsPerServer; + } + + @Config("max-connections-per-server") + public PinotConfig setMaxConnectionsPerServer(String maxConnectionsPerServer) + { + try { + this.maxConnectionsPerServer = Integer.valueOf(maxConnectionsPerServer); + } + catch (Exception e) { + this.maxConnectionsPerServer = DEFAULT_MAX_CONNECTIONS_PER_SERVER; + } + return this; + } + + @NotNull + public int getMaxBacklogPerServer() + { + return maxBacklogPerServer; + } + + @Config("max-backlog-per-server") + public PinotConfig setMaxBacklogPerServer(String maxBacklogPerServer) + { + try { + this.maxBacklogPerServer = Integer.valueOf(maxBacklogPerServer); + } + catch (Exception e) { + this.maxBacklogPerServer = DEFAULT_MAX_BACKLOG_PER_SERVER; + } + return this; + } + + @MinDuration("15s") + @NotNull + public Duration getIdleTimeout() + { + return idleTimeout; + } + + @Config("idle-timeout") + public PinotConfig setIdleTimeout(Duration idleTimeout) + { + this.idleTimeout = idleTimeout; + return this; + } + + @MinDuration("15s") + @NotNull + public Duration getConnectionTimeout() + { + return connectionTimeout; + } + + @Config("connection-timeout") + public PinotConfig setConnectionTimeout(Duration connectionTimeout) + { + this.connectionTimeout = connectionTimeout; + return this; + } + + @NotNull + public int getEstimatedSizeInBytesForNonNumericColumn() + { + return estimatedSizeInBytesForNonNumericColumn; + } + + @Config("estimated-size-in-bytes-for-non-numeric-column") + public PinotConfig setEstimatedSizeInBytesForNonNumericColumn(int estimatedSizeInBytesForNonNumericColumn) + { + try { + this.estimatedSizeInBytesForNonNumericColumn = Integer.valueOf(estimatedSizeInBytesForNonNumericColumn); + } + catch (Exception e) { + this.estimatedSizeInBytesForNonNumericColumn = DEFAULT_ESTIMATED_SIZE_IN_BYTES_FOR_NON_NUMERIC_COLUMN; + } + return this; + } +} diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotConnection.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotConnection.java new file mode 100644 index 0000000000000..f25ffbe93a587 --- /dev/null +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotConnection.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.PrestoException; +import com.google.inject.Inject; +import io.airlift.log.Logger; +import org.apache.pinot.common.data.Schema; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import static com.facebook.presto.pinot.PinotErrorCode.PINOT_FAILURE_GETTING_SCHEMA; +import static com.facebook.presto.pinot.PinotErrorCode.PINOT_FAILURE_GETTING_TABLE; + +public class PinotConnection +{ + private static final Logger log = Logger.get(PinotConnection.class); + + private final PinotClusterInfoFetcher pinotClusterInfoFetcher; + + @Inject + public PinotConnection(PinotClusterInfoFetcher pinotClusterInfoFetcher) + { + this.pinotClusterInfoFetcher = pinotClusterInfoFetcher; + } + + public Map>> getRoutingTable(String table) + throws Exception + { + Map>> routingTableForTable = this.pinotClusterInfoFetcher.getRoutingTableForTable(table); + log.debug("RoutingTable for table: %s is %s", table, Arrays.toString(routingTableForTable.entrySet().toArray())); + return routingTableForTable; + } + + public Map getTimeBoundary(String table) + throws Exception + { + Map timeBoundaryForTable = this.pinotClusterInfoFetcher.getTimeBoundaryForTable(table); + log.debug("TimeBoundary for table: %s is %s", table, Arrays.toString(timeBoundaryForTable.entrySet().toArray())); + return timeBoundaryForTable; + } + + public List getTableNames() + { + try { + return pinotClusterInfoFetcher.getAllTables(); + } + catch (Exception e) { + throw new PrestoException( + PINOT_FAILURE_GETTING_TABLE, + "Failed getting table names from Pinot", + e); + } + } + + public PinotTable getTable(String tableName) + { + List columns = getPinotColumnsForTable(tableName); + return new PinotTable(tableName, columns); + } + + private Schema getPinotTableSchema(String tableName) + { + try { + return pinotClusterInfoFetcher.getTableSchema(tableName); + } + catch (Exception e) { + throw new PrestoException( + PINOT_FAILURE_GETTING_SCHEMA, + String.format("Failed getting schema for %s", tableName), + e); + } + } + + private List getPinotColumnsForTable(String tableName) + { + Schema pinotTableSchema = getPinotTableSchema(tableName); + return PinotColumnUtils.getPinotColumnsForPinotSchema(pinotTableSchema); + } + + public PinotColumn getPinotTimeColumnForTable(String tableName) + { + Schema pinotTableSchema = getPinotTableSchema(tableName); + String columnName = pinotTableSchema.getTimeColumnName(); + return new PinotColumn(columnName, PinotColumnUtils.getPrestoTypeFromPinotType(pinotTableSchema.getFieldSpecFor(columnName))); + } +} diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotConnector.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotConnector.java new file mode 100644 index 0000000000000..e78c07de9e9d0 --- /dev/null +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotConnector.java @@ -0,0 +1,83 @@ +/* + * 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.Connector; +import com.facebook.presto.spi.connector.ConnectorMetadata; +import com.facebook.presto.spi.connector.ConnectorPageSourceProvider; +import com.facebook.presto.spi.connector.ConnectorSplitManager; +import com.facebook.presto.spi.connector.ConnectorTransactionHandle; +import com.facebook.presto.spi.transaction.IsolationLevel; +import io.airlift.bootstrap.LifeCycleManager; +import io.airlift.log.Logger; + +import javax.inject.Inject; + +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; + + @Inject + public PinotConnector(LifeCycleManager lifeCycleManager, PinotMetadata metadata, PinotSplitManager splitManager, PinotPageSourceProvider pageSourceProvider) + { + 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"); + } + + @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 final void shutdown() + { + try { + lifeCycleManager.stop(); + } + catch (Exception e) { + log.error(e, "Error shutting down connector"); + } + } +} diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotConnectorFactory.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotConnectorFactory.java new file mode 100644 index 0000000000000..82e4b8887e9f8 --- /dev/null +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotConnectorFactory.java @@ -0,0 +1,67 @@ +/* + * 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.ConnectorHandleResolver; +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.type.TypeManager; +import com.google.common.base.Throwables; +import com.google.inject.Injector; +import io.airlift.bootstrap.Bootstrap; +import io.airlift.json.JsonModule; + +import java.util.Map; + +import static java.util.Objects.requireNonNull; + +public class PinotConnectorFactory + implements ConnectorFactory +{ + @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 PinotModule(), binder -> { + binder.bind(PinotConnectorId.class).toInstance(new PinotConnectorId(connectorId)); + binder.bind(TypeManager.class).toInstance(context.getTypeManager()); + binder.bind(NodeManager.class).toInstance(context.getNodeManager()); + }); + + Injector injector = app.strictConfig().doNotInitializeLogging().setRequiredConfigurationProperties(config).initialize(); + + return injector.getInstance(PinotConnector.class); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } +} diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotConnectorId.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotConnectorId.java new file mode 100644 index 0000000000000..d9b1e03b38697 --- /dev/null +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotConnectorId.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 java.util.Objects; + +import static java.util.Objects.requireNonNull; + +public final class PinotConnectorId +{ + private final String id; + + public PinotConnectorId(String id) + { + this.id = requireNonNull(id, "id is null"); + } + + @Override + public String toString() + { + return id; + } + + @Override + public int hashCode() + { + return Objects.hash(id); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if ((obj == null) || (getClass() != obj.getClass())) { + return false; + } + + PinotConnectorId other = (PinotConnectorId) obj; + return Objects.equals(this.id, other.id); + } +} diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotErrorCode.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotErrorCode.java new file mode 100644 index 0000000000000..b110193b598ae --- /dev/null +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotErrorCode.java @@ -0,0 +1,49 @@ +/* + * 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; + +public enum PinotErrorCode + implements ErrorCodeSupplier +{ + PINOT_UNSUPPORTED_COLUMN_TYPE(0, EXTERNAL), + PINOT_FAILURE_GETTING_TABLE(1, EXTERNAL), + PINOT_FAILURE_GETTING_SCHEMA(2, EXTERNAL), + PINOT_FAILURE_QUERYING_DATA(3, 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; + + PinotErrorCode(int code, ErrorType type) + { + errorCode = new ErrorCode(code + 0x0505_0000, name(), type); + } + + @Override + public ErrorCode toErrorCode() + { + return errorCode; + } +} diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotHandleResolver.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotHandleResolver.java new file mode 100644 index 0000000000000..d4e462b77d01b --- /dev/null +++ b/presto-pinot/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/src/main/java/com/facebook/presto/pinot/PinotMetadata.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotMetadata.java new file mode 100644 index 0000000000000..5cf1a1f2b06b6 --- /dev/null +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotMetadata.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.spi.ColumnHandle; +import com.facebook.presto.spi.ColumnMetadata; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.ConnectorTableHandle; +import com.facebook.presto.spi.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.PrestoException; +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 com.google.common.collect.ImmutableSet; +import io.airlift.log.Logger; + +import javax.inject.Inject; + +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static com.facebook.presto.pinot.PinotErrorCode.PINOT_FAILURE_GETTING_TABLE; +import static com.facebook.presto.pinot.Types.checkType; +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 static final Logger log = Logger.get(PinotMetadata.class); + + private final String connectorId; + private final PinotConnection pinotPrestoConnection; + + @Inject + public PinotMetadata(PinotConnectorId 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 listSchemaNames(); + } + + public List listSchemaNames() + { + try { + ImmutableList.Builder schemaNamesListBuilder = ImmutableList.builder(); + for (String table : pinotPrestoConnection.getTableNames()) { + schemaNamesListBuilder.add(table.toLowerCase(ENGLISH)); + } + return schemaNamesListBuilder.build(); + } + catch (Exception e) { + return ImmutableList.of(); + } + } + + public String getPinotTableNameFromPrestoTableName(String prestoTableName) + { + for (String pinotTableName : pinotPrestoConnection.getTableNames()) { + if (prestoTableName.equalsIgnoreCase(pinotTableName)) { + return pinotTableName; + } + } + return null; + } + + @Override + public PinotTableHandle getTableHandle(ConnectorSession session, SchemaTableName tableName) + { + if (!listSchemaNames(session).contains(tableName.getSchemaName().toLowerCase(ENGLISH))) { + return null; + } + String pinotTableName; + try { + pinotTableName = getPinotTableNameFromPrestoTableName(tableName.getTableName()); + PinotTable table = pinotPrestoConnection.getTable(pinotTableName); + if (table == null) { + return null; + } + } + catch (Exception e) { + throw new PrestoException( + PINOT_FAILURE_GETTING_TABLE, + String.format("Failed to get TableHandle for %s", tableName), + e); + } + return new PinotTableHandle(connectorId, tableName.getSchemaName(), pinotTableName); + } + + @Override + public List getTableLayouts(ConnectorSession session, ConnectorTableHandle table, Constraint constraint, Optional> desiredColumns) + { + PinotTableHandle tableHandle = checkType(table, PinotTableHandle.class, "table"); + tableHandle.setConstraintSummary(constraint.getSummary()); + ConnectorTableLayout layout = new ConnectorTableLayout(new PinotTableLayoutHandle(tableHandle)); + 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 = checkType(table, PinotTableHandle.class, "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) + { + Collection schemaNames; + if (schemaNameOrNull != null) { + schemaNames = ImmutableSet.of(schemaNameOrNull); + } + else { + schemaNames = listSchemaNames(); + } + + ImmutableList.Builder builder = ImmutableList.builder(); + try { + for (String table : pinotPrestoConnection.getTableNames()) { + if (schemaNames.contains(table.toLowerCase(ENGLISH))) { + builder.add(new SchemaTableName(table.toLowerCase(ENGLISH), table)); + } + } + } + catch (Exception e) { + } + return builder.build(); + } + + @Override + public Map getColumnHandles(ConnectorSession session, ConnectorTableHandle tableHandle) + { + PinotTableHandle pinotTableHandle = checkType(tableHandle, PinotTableHandle.class, "tableHandle"); + checkArgument(pinotTableHandle.getConnectorId().equals(connectorId), "tableHandle is not for this connector"); + + String pinotTableName = getPinotTableNameFromPrestoTableName(pinotTableHandle.getTableName()); + try { + PinotTable table = pinotPrestoConnection.getTable(pinotTableName); + if (table == null) { + throw new TableNotFoundException(pinotTableHandle.toSchemaTableName()); + } + ImmutableMap.Builder columnHandles = ImmutableMap.builder(); + int index = 0; + for (ColumnMetadata column : table.getColumnsMetadata()) { + columnHandles.put(column.getName().toLowerCase(ENGLISH), new PinotColumnHandle(connectorId, ((PinotColumnMetadata) column).getPinotName(), column.getType(), index)); + index++; + } + return columnHandles.build(); + } + catch (Exception e) { + log.error("Failed to get ColumnHandles for table : " + pinotTableHandle.getTableName(), e); + return null; + } + } + + @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) + { + if (!listSchemaNames().contains(tableName.getSchemaName())) { + return null; + } + 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) { + return listTables(session, prefix.getSchemaName()); + } + return ImmutableList.of(new SchemaTableName(prefix.getSchemaName(), prefix.getTableName())); + } + + @Override + public ColumnMetadata getColumnMetadata(ConnectorSession session, ConnectorTableHandle tableHandle, ColumnHandle columnHandle) + { + checkType(tableHandle, PinotTableHandle.class, "tableHandle"); + return checkType(columnHandle, PinotColumnHandle.class, "columnHandle").getColumnMetadata(); + } +} diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotModule.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotModule.java new file mode 100644 index 0000000000000..fb1dc03bfd15e --- /dev/null +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotModule.java @@ -0,0 +1,77 @@ +/* + * 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.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 javax.inject.Inject; + +import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature; +import static com.google.common.base.Preconditions.checkArgument; +import static io.airlift.configuration.ConfigBinder.configBinder; +import static io.airlift.json.JsonBinder.jsonBinder; +import static io.airlift.json.JsonCodec.listJsonCodec; +import static io.airlift.json.JsonCodecBinder.jsonCodecBinder; +import static java.util.Objects.requireNonNull; + +/** + * Guice module for the Pinot connector. + */ +public class PinotModule + implements Module +{ + @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(PinotSplitManager.class).in(Scopes.SINGLETON); + binder.bind(PinotPageSourceProvider.class).in(Scopes.SINGLETON); + binder.bind(PinotScatterGatherQueryClient.class).in(Scopes.SINGLETON); + binder.bind(PinotClusterInfoFetcher.class).in(Scopes.SINGLETON); + binder.bind(PinotConnection.class).in(Scopes.SINGLETON); + + jsonBinder(binder).addDeserializerBinding(Type.class).to(TypeDeserializer.class); + jsonCodecBinder(binder).bindMapJsonCodec(String.class, listJsonCodec(PinotTable.class)); + } + + @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/src/main/java/com/facebook/presto/pinot/PinotPageSource.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotPageSource.java new file mode 100644 index 0000000000000..9d41461cfe2af --- /dev/null +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotPageSource.java @@ -0,0 +1,399 @@ +/* + * 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.ConnectorPageSource; +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.ImmutableList; +import io.airlift.log.Logger; +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.Arrays; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.stream.IntStream; + +import static com.facebook.presto.pinot.PinotErrorCode.PINOT_UNSUPPORTED_COLUMN_TYPE; +import static com.facebook.presto.pinot.PinotQueryGenerator.getPinotQuery; +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 PinotPageSource + implements ConnectorPageSource +{ + private static final Logger log = Logger.get(PinotPageSource.class); + + private List columnHandles; + private List columnTypes; + + private PinotConfig pinotConfig; + private PinotSplit split; + private PinotScatterGatherQueryClient pinotQueryClient; + + // 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; + // Stores the mapping between pinot column name and the column index + Map pinotColumnNameIndexMap = new HashMap<>(); + + public PinotPageSource(PinotConfig pinotConfig, PinotScatterGatherQueryClient pinotQueryClient, PinotSplit split, List columnHandles) + { + requireNonNull(split, "split is null"); + 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"); + } + + @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()); + } + + /** + * Iterate through each Pinot {@link org.apache.pinot.common.utils.DataTable}, and construct a {@link com.facebook.presto.spi.Page} out of it. + * + * @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); + writeBlock(blockBuilder, columnType, pinotColumnNameIndexMap.get(columnHandles.get(columnHandleIdx).getColumnName())); + } + Page page = pageBuilder.build(); + return page; + } + + /** + * Fetch data from Pinot for the current split and store the {@link org.apache.pinot.common.utils.DataTable} returned from each Pinto server. + */ + private void fetchPinotData() + { + log.debug("Fetching data from Pinot for table %s, segment %s", split.getTableName(), split.getSegment()); + long startTimeNanos = System.nanoTime(); + int idx = 0; + for (PinotColumnHandle columnHandle : columnHandles) { + pinotColumnNameIndexMap.put(columnHandle.getColumnName(), idx++); + } + String pinotQuery = getPinotQuery(pinotConfig, columnHandles, split.getPinotFilter(), split.getTimeFilter(), split.getTableName(), split.getLimit()); + Map dataTableMap = pinotQueryClient.queryPinotServerForDataTable(pinotQuery, split.getHost(), split.getSegment()); + dataTableMap.values().stream() + // ignore empty tables and tables with 0 rows + .filter(table -> table != null && table.getNumberOfRows() > 0) + .forEach(dataTable -> + { + // 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; + }); + ImmutableList.Builder types = ImmutableList.builder(); + columnHandles + .stream() + .map(columnHandle -> getTypeForBlock(columnHandle)) + .forEach(types::add); + this.columnTypes = types.build(); + readTimeNanos = System.nanoTime() - startTimeNanos; + isPinotDataFetched = true; + } + + @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 columnIdx) + { + IntStream.rangeClosed(0, currentDataTable.getDataTable().getNumberOfRows() - 1).forEach(i -> + { + columnType.writeBoolean(blockBuilder, getBoolean(i, columnIdx)); + completedBytes++; + }); + } + + private void writeLongBlock(BlockBuilder blockBuilder, Type columnType, int columnIdx) + { + IntStream.rangeClosed(0, currentDataTable.getDataTable().getNumberOfRows() - 1).forEach(i -> + { + columnType.writeLong(blockBuilder, getLong(i, columnIdx)); + completedBytes += Long.BYTES; + }); + } + + private void writeDoubleBlock(BlockBuilder blockBuilder, Type columnType, int columnIdx) + { + IntStream.rangeClosed(0, currentDataTable.getDataTable().getNumberOfRows() - 1).forEach(i -> + { + columnType.writeDouble(blockBuilder, getDouble(i, columnIdx)); + completedBytes += Double.BYTES; + }); + } + + private void writeSliceBlock(BlockBuilder blockBuilder, Type columnType, int columnIdx) + { + IntStream.rangeClosed(0, currentDataTable.getDataTable().getNumberOfRows() - 1).forEach(i -> + { + Slice slice = getSlice(i, columnIdx); + columnType.writeSlice(blockBuilder, slice, 0, slice.length()); + completedBytes += slice.getBytes().length; + }); + } + + Type getType(int colIdx) + { + checkArgument(colIdx < columnHandles.size(), "Invalid field index"); + return columnHandles.get(colIdx).getColumnType(); + } + + boolean getBoolean(int rowIdx, int colIdx) + { + return Boolean.getBoolean(currentDataTable.getDataTable().getString(rowIdx, colIdx)); + } + + long getLong(int rowIdx, int colIdx) + { + DataSchema.ColumnDataType dataType = currentDataTable.getDataTable().getDataSchema().getColumnDataType(colIdx); + // 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(rowIdx, colIdx); + } + if (dataType.equals(DataType.INT)) { + return (long) currentDataTable.getDataTable().getInt(rowIdx, colIdx); + } + else { + return currentDataTable.getDataTable().getLong(rowIdx, colIdx); + } + } + + double getDouble(int rowIdx, int colIdx) + { + DataSchema.ColumnDataType dataType = currentDataTable.getDataTable().getDataSchema().getColumnDataType(colIdx); + if (dataType.equals(DataType.FLOAT)) { + return currentDataTable.getDataTable().getFloat(rowIdx, colIdx); + } + else { + return currentDataTable.getDataTable().getDouble(rowIdx, colIdx); + } + } + + Slice getSlice(int rowIdx, int colIdx) + { + checkColumnType(colIdx, VARCHAR); + DataSchema.ColumnDataType columnType = currentDataTable.getDataTable().getDataSchema().getColumnDataType(colIdx); + switch (columnType) { + case INT_ARRAY: + int[] intArray = currentDataTable.getDataTable().getIntArray(rowIdx, colIdx); + return utf8Slice(Arrays.toString(intArray)); + case LONG_ARRAY: + long[] longArray = currentDataTable.getDataTable().getLongArray(rowIdx, colIdx); + return utf8Slice(Arrays.toString(longArray)); + case FLOAT_ARRAY: + float[] floatArray = currentDataTable.getDataTable().getFloatArray(rowIdx, colIdx); + return utf8Slice(Arrays.toString(floatArray)); + case DOUBLE_ARRAY: + double[] doubleArray = currentDataTable.getDataTable().getDoubleArray(rowIdx, colIdx); + return utf8Slice(Arrays.toString(doubleArray)); + case STRING_ARRAY: + String[] stringArray = currentDataTable.getDataTable().getStringArray(rowIdx, colIdx); + return utf8Slice(Arrays.toString(stringArray)); + case STRING: + String fieldStr = currentDataTable.getDataTable().getString(rowIdx, colIdx); + if (fieldStr == null || fieldStr.isEmpty()) { + return Slices.EMPTY_SLICE; + } + return Slices.utf8Slice(fieldStr); + } + 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; + } + } + else { + return pinotConfig.getEstimatedSizeInBytesForNonNumericColumn(); + } + } + + void checkColumnType(int colIdx, Type expected) + { + Type actual = getType(colIdx); + checkArgument(actual.equals(expected), "Expected column %s to be type %s but is %s", colIdx, expected, actual); + } + + Type getTypeForBlock(PinotColumnHandle pinotColumnHandle) + { + if (pinotColumnHandle.getColumnType().equals(INTEGER)) { + return BIGINT; + } + else { + return pinotColumnHandle.getColumnType(); + } + } + + private 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/src/main/java/com/facebook/presto/pinot/PinotPageSourceProvider.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotPageSourceProvider.java new file mode 100644 index 0000000000000..dda39132d9dd7 --- /dev/null +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotPageSourceProvider.java @@ -0,0 +1,67 @@ +/* + * 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.ConnectorPageSource; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.ConnectorSplit; +import com.facebook.presto.spi.connector.ConnectorPageSourceProvider; +import com.facebook.presto.spi.connector.ConnectorTransactionHandle; + +import javax.inject.Inject; + +import java.util.ArrayList; +import java.util.List; + +import static com.facebook.presto.pinot.Types.checkType; +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; + + @Inject + public PinotPageSourceProvider(PinotConnectorId connectorId, PinotConfig pinotConfig, PinotScatterGatherQueryClient pinotQueryClient) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); + this.pinotConfig = requireNonNull(pinotConfig, "pinotConfig is null"); + this.pinotQueryClient = requireNonNull(pinotQueryClient, "pinotQueryClient is null"); + } + + @Override + public ConnectorPageSource createPageSource(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorSplit split, List columns) + { + requireNonNull(split, "partitionChunk is null"); + PinotSplit pinotSplit = checkType(split, PinotSplit.class, "split"); + checkArgument(pinotSplit.getConnectorId().equals(connectorId), "split is not for this connector"); + + List handles = new ArrayList<>(); + if (columns.isEmpty()) { + // For COUNT(*) and COUNT(1), no columns are passed down to Pinot + // Since this is the only known type of queries for this scenario, we just select time column from Pinot to facilitate the COUNT + handles.add(new PinotColumnHandle(this.connectorId, pinotSplit.getTimeColumn().getName(), pinotSplit.getTimeColumn().getType(), 0)); + } + else { + for (ColumnHandle handle : columns) { + handles.add(checkType(handle, PinotColumnHandle.class, "handle")); + } + } + return new PinotPageSource(this.pinotConfig, this.pinotQueryClient, pinotSplit, handles); + } +} 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..d2875462b73a5 --- /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 synchronized Iterable getConnectorFactories() + { + return ImmutableList.of(new PinotConnectorFactory()); + } +} diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotQueryGenerator.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotQueryGenerator.java new file mode 100644 index 0000000000000..8bbaf5aa34149 --- /dev/null +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotQueryGenerator.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 io.airlift.log.Logger; + +import java.util.List; +import java.util.StringJoiner; + +import static java.util.Objects.requireNonNull; + +/** + * This class manages how to generate the query to send to Pinot servers. + */ +public final class PinotQueryGenerator +{ + private static final Logger log = Logger.get(PinotQueryGenerator.class); + + private PinotQueryGenerator() + { + } + + /** + * QUERY_TEMPLATE looks like this: + * SELECT $fields FROM $tableName $predicate LIMIT $limit. + *

+ * Note $predicate is optional, and we intentionally add a space between $tableName $predicate for readability. + * When $predicate is absent, there would be 2 spaces between $tableName and LIMIT, which is should not hurt the query itself. + */ + public static final String QUERY_TEMPLATE = "SELECT %s FROM %s %s LIMIT %d"; + + /** + * Returns the Pinot Query to send for each split. + * + *

Pinot Query would be constructed based on {$link #QUERY_TEMPLATE} and predicates (WHERE ...). + * + * @return the constructed Pinot Query + */ + static String getPinotQuery(PinotConfig pinotConfig, List columnHandles, String pinotFilter, String timeFilter, String tableName, long splitLimit) + { + requireNonNull(pinotConfig, "pinotConfig is null"); + StringJoiner fieldsJoiner = new StringJoiner(", "); + for (PinotColumnHandle columnHandle : columnHandles) { + // No aggregation pushdown + fieldsJoiner.add(columnHandle.getColumnName()); + } + + // Add predicates + StringJoiner predicatesJoiner = new StringJoiner(" AND "); + if (!pinotFilter.isEmpty()) { + predicatesJoiner.add(String.format("(%s)", pinotFilter)); + } + if (!timeFilter.isEmpty()) { + predicatesJoiner.add(String.format("(%s)", timeFilter)); + } + + // Note pinotPredicate is optional. It would be empty when no predicates are pushed down. + String pinotPredicate = ""; + if (predicatesJoiner.length() > 0) { + pinotPredicate = "WHERE " + predicatesJoiner.toString(); + } + + long limit = splitLimit > 0 ? splitLimit : pinotConfig.getLimitAll(); + + final String finalQuery = String.format(QUERY_TEMPLATE, fieldsJoiner.toString(), tableName, pinotPredicate, limit); + log.debug("Plan to send PQL : %s", finalQuery); + return finalQuery; + } +} diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotScatterGatherQueryClient.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotScatterGatherQueryClient.java new file mode 100644 index 0000000000000..84dc9dc174c2c --- /dev/null +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotScatterGatherQueryClient.java @@ -0,0 +1,319 @@ +/* + * 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 com.google.inject.Inject; +import com.yammer.metrics.core.MetricsRegistry; +import io.airlift.log.Logger; +import io.airlift.units.Duration; +import io.netty.channel.EventLoopGroup; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.util.HashedWheelTimer; +import org.apache.pinot.common.exception.QueryException; +import org.apache.pinot.common.metrics.BrokerMeter; +import org.apache.pinot.common.metrics.BrokerMetrics; +import org.apache.pinot.common.request.BrokerRequest; +import org.apache.pinot.common.request.InstanceRequest; +import org.apache.pinot.common.response.ProcessingException; +import org.apache.pinot.common.response.ServerInstance; +import org.apache.pinot.common.utils.DataTable; +import org.apache.pinot.core.common.datatable.DataTableFactory; +import org.apache.pinot.pql.parsers.Pql2Compiler; +import org.apache.pinot.serde.SerDe; +import org.apache.pinot.transport.common.CompositeFuture; +import org.apache.pinot.transport.metrics.NettyClientMetrics; +import org.apache.pinot.transport.netty.PooledNettyClientResourceManager; +import org.apache.pinot.transport.pool.KeyedPool; +import org.apache.pinot.transport.pool.KeyedPoolImpl; +import org.apache.pinot.transport.scattergather.ScatterGather; +import org.apache.pinot.transport.scattergather.ScatterGatherImpl; +import org.apache.pinot.transport.scattergather.ScatterGatherRequest; +import org.apache.pinot.transport.scattergather.ScatterGatherStats; +import org.apache.thrift.protocol.TCompactProtocol; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.atomic.AtomicLong; + +import static com.facebook.presto.pinot.PinotErrorCode.PINOT_FAILURE_QUERYING_DATA; + +/** + * This class acts as the Pinot broker, fetches data from Pinot segments, gathers and returns the result. + * Many components were taken from ConnectionPoolBrokerRequestHandler + */ +public class PinotScatterGatherQueryClient +{ + private static final Logger log = Logger.get(PinotScatterGatherQueryClient.class); + + private static final Pql2Compiler REQUEST_COMPILER = new Pql2Compiler(); + private static final String PRESTO_HOST_PREFIX = "presto-pinot-master"; + private static final boolean DEFAULT_EMIT_TABLE_LEVEL_METRICS = true; + + private final AtomicLong requestIdGenerator; + private final String prestoHostId; + private final MetricsRegistry registry; + private final BrokerMetrics brokerMetrics; + private final ScatterGather scatterGatherer; + // Netty Specific + private EventLoopGroup eventLoopGroup; + private PooledNettyClientResourceManager resourceManager; + // Connection Pool Related + private KeyedPool connPool; + private ScheduledThreadPoolExecutor poolTimeoutExecutor; + private ExecutorService requestSenderPool; + private Duration connectionTimeout; + + @Inject + public PinotScatterGatherQueryClient(PinotConfig pinotConfig) + { + requestIdGenerator = new AtomicLong(0); + prestoHostId = getDefaultPrestoId(); + + registry = new MetricsRegistry(); + brokerMetrics = new BrokerMetrics(registry, DEFAULT_EMIT_TABLE_LEVEL_METRICS); + brokerMetrics.initializeGlobalMeters(); + + eventLoopGroup = new NioEventLoopGroup(); + /** + * Some of the client metrics uses histogram which is doing synchronous operation. + * These are fixed overhead per request/response. + * TODO: Measure the overhead of this. + */ + final NettyClientMetrics clientMetrics = new NettyClientMetrics(registry, "presto_pinot_client_"); + + // Setup Netty Connection Pool + resourceManager = new PooledNettyClientResourceManager(eventLoopGroup, new HashedWheelTimer(), clientMetrics); + + requestSenderPool = Executors.newFixedThreadPool(pinotConfig.getThreadPoolSize()); + poolTimeoutExecutor = new ScheduledThreadPoolExecutor(pinotConfig.getCorePoolSize()); + connectionTimeout = pinotConfig.getConnectionTimeout(); + connPool = new KeyedPoolImpl(pinotConfig.getMinConnectionsPerServer(), pinotConfig.getMaxConnectionsPerServer(), pinotConfig.getIdleTimeout().toMillis(), pinotConfig.getMaxBacklogPerServer(), resourceManager, poolTimeoutExecutor, requestSenderPool, registry); + resourceManager.setPool(connPool); + + // Setup ScatterGather + scatterGatherer = new ScatterGatherImpl(connPool, requestSenderPool); + } + + private String getDefaultPrestoId() + { + String defaultBrokerId; + try { + defaultBrokerId = PRESTO_HOST_PREFIX + InetAddress.getLocalHost().getHostName(); + } + catch (UnknownHostException e) { + log.error("Caught exception while getting default broker id", e); + defaultBrokerId = PRESTO_HOST_PREFIX; + } + return defaultBrokerId; + } + + public Map queryPinotServerForDataTable(String pql, String serverHost, String segment) + { + long requestId = requestIdGenerator.incrementAndGet(); + BrokerRequest brokerRequest; + try { + brokerRequest = REQUEST_COMPILER.compileToBrokerRequest(pql); + } + catch (Exception e) { + throw new PrestoException( + PINOT_FAILURE_QUERYING_DATA, + String.format("Parsing error on requestId %d, PQL = %s", requestId, pql), + e); + } + + Map> routingTable = new HashMap<>(); + List segmentList = new ArrayList<>(); + segmentList.add(segment); + routingTable.put(serverHost, segmentList); + ScatterGatherRequestImpl scatterRequest = new ScatterGatherRequestImpl(brokerRequest, routingTable, 0, connectionTimeout.toMillis(), prestoHostId); + + ScatterGatherStats scatterGatherStats = new ScatterGatherStats(); + CompositeFuture compositeFuture = routeScatterGather(scatterRequest, scatterGatherStats); + + if (compositeFuture == null) { + // No server found in either OFFLINE or REALTIME table. + return null; + } + + Map dataTableMap = new HashMap<>(); + + List processingExceptions = new ArrayList<>(); + Map serverResponseMap = null; + serverResponseMap = gatherServerResponses(compositeFuture, scatterGatherStats, true, brokerRequest.getQuerySource().getTableName(), processingExceptions); + deserializeServerResponses(serverResponseMap, true, dataTableMap, brokerRequest.getQuerySource().getTableName(), processingExceptions); + return dataTableMap; + } + + /** + * Gather responses from servers, append processing exceptions to the processing exception list passed in. + * + * @param compositeFuture composite future returned from scatter phase. + * @param scatterGatherStats scatter-gather statistics. + * @param isOfflineTable whether the scatter-gather target is an OFFLINE table. + * @param tableNameWithType table name with type suffix. + * @param processingExceptions list of processing exceptions. + * @return server response map. + */ + @Nullable + private Map gatherServerResponses( + @Nonnull CompositeFuture compositeFuture, + @Nonnull ScatterGatherStats scatterGatherStats, boolean isOfflineTable, + @Nonnull String tableNameWithType, + @Nonnull List processingExceptions) + { + try { + Map serverResponseMap = compositeFuture.get(); + Iterator> iterator = serverResponseMap.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry entry = iterator.next(); + if (entry.getValue().length == 0) { + throw new PrestoException( + PINOT_FAILURE_QUERYING_DATA, + String.format("Got empty data for table: %s in server %s.", tableNameWithType, entry.getKey().getShortHostName())); + } + } + Map responseTimes = compositeFuture.getResponseTimes(); + scatterGatherStats.setResponseTimeMillis(responseTimes, isOfflineTable); + return serverResponseMap; + } + catch (Exception e) { + brokerMetrics.addMeteredTableValue(tableNameWithType, BrokerMeter.RESPONSE_FETCH_EXCEPTIONS, 1L); + processingExceptions.add(QueryException.getException(QueryException.BROKER_GATHER_ERROR, e)); + throw new PrestoException( + PINOT_FAILURE_QUERYING_DATA, + String.format("Caught exception while fetching responses for table: %s", tableNameWithType), + e); + } + } + + /** + * Deserialize the server responses, put the de-serialized data table into the data table map passed in, append + * processing exceptions to the processing exception list passed in. + *

For hybrid use case, multiple responses might be from the same instance. Use response sequence to distinguish + * them. + * + * @param responseMap map from server to response. + * @param isOfflineTable whether the responses are from an OFFLINE table. + * @param dataTableMap map from server to data table. + * @param tableNameWithType table name with type suffix. + * @param processingExceptions list of processing exceptions. + */ + private void deserializeServerResponses( + @Nonnull Map responseMap, boolean isOfflineTable, + @Nonnull Map dataTableMap, + @Nonnull String tableNameWithType, + @Nonnull List processingExceptions) + { + for (Map.Entry entry : responseMap.entrySet()) { + ServerInstance serverInstance = entry.getKey(); + if (!isOfflineTable) { + serverInstance = serverInstance.withSeq(1); + } + try { + dataTableMap.put(serverInstance, DataTableFactory.getDataTable(entry.getValue())); + } + catch (Exception e) { + brokerMetrics.addMeteredTableValue(tableNameWithType, BrokerMeter.DATA_TABLE_DESERIALIZATION_EXCEPTIONS, 1L); + processingExceptions.add(QueryException.getException(QueryException.DATA_TABLE_DESERIALIZATION_ERROR, e)); + throw new PrestoException( + PINOT_FAILURE_QUERYING_DATA, + String.format("Caught exceptions while deserializing response for table: %s from server: %s", tableNameWithType, serverInstance), + e); + } + } + } + + private CompositeFuture routeScatterGather(ScatterGatherRequestImpl scatterRequest, ScatterGatherStats scatterGatherStats) + { + CompositeFuture compositeFuture = null; + try { + compositeFuture = this.scatterGatherer.scatterGather(scatterRequest, scatterGatherStats, true, brokerMetrics); + } + catch (InterruptedException e) { + throw new PrestoException( + PINOT_FAILURE_QUERYING_DATA, + "Caught exception querying Pinot servers.", + e); + } + return compositeFuture; + } + + private static class ScatterGatherRequestImpl + implements ScatterGatherRequest + { + private final BrokerRequest brokerRequest; + private final Map> routingTable; + private final long requestId; + private final long requestTimeoutMs; + private final String brokerId; + + public ScatterGatherRequestImpl(BrokerRequest request, Map> routingTable, long requestId, long requestTimeoutMs, String brokerId) + { + brokerRequest = request; + this.routingTable = routingTable; + this.requestId = requestId; + + this.requestTimeoutMs = requestTimeoutMs; + this.brokerId = brokerId; + } + + @Override + public Map> getRoutingTable() + { + return routingTable; + } + + @Override + public byte[] getRequestForService(List segments) + { + InstanceRequest r = new InstanceRequest(); + r.setRequestId(requestId); + r.setEnableTrace(brokerRequest.isEnableTrace()); + r.setQuery(brokerRequest); + r.setSearchSegments(segments); + r.setBrokerId(brokerId); + return new SerDe(new TCompactProtocol.Factory()).serialize(r); + } + + @Override + public long getRequestId() + { + return requestId; + } + + @Override + public long getRequestTimeoutMs() + { + return requestTimeoutMs; + } + + @Override + public BrokerRequest getBrokerRequest() + { + return brokerRequest; + } + } +} diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotSplit.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotSplit.java new file mode 100644 index 0000000000000..7e2b6069de775 --- /dev/null +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotSplit.java @@ -0,0 +1,131 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.pinot; + +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 io.airlift.log.Logger; + +import java.util.List; + +import static java.util.Objects.requireNonNull; + +public class PinotSplit + implements ConnectorSplit +{ + private static final Logger log = Logger.get(PinotSplit.class); + + private final String connectorId; + private final String tableName; + private final String host; + private final String segment; + private final boolean remotelyAccessible; + private final List addresses; + private final PinotColumn timeColumn; + private final String timeFilter; + private final String pinotFilter; + private final long limit; + + @JsonCreator + public PinotSplit( + @JsonProperty("connectorId") String connectorId, + @JsonProperty("tableName") String tableName, + @JsonProperty("host") String host, + @JsonProperty("segment") String segment, + @JsonProperty("timeColumn") PinotColumn timeColumn, + @JsonProperty("timeFilter") String timeFilter, + @JsonProperty("pinotFilter") String pinotFilter, + @JsonProperty("limit") long limit) + { + this.connectorId = requireNonNull(connectorId, "connector id is null"); + this.tableName = requireNonNull(tableName, "table name is null"); + this.host = requireNonNull(host, "host is null"); + this.segment = requireNonNull(segment, "segment is null"); + this.timeColumn = requireNonNull(timeColumn, "timeColumn is null"); + this.addresses = null; + this.pinotFilter = pinotFilter; + this.timeFilter = timeFilter; + this.limit = limit; + this.remotelyAccessible = true; + } + + @JsonProperty + public String getConnectorId() + { + return connectorId; + } + + @JsonProperty + public String getTableName() + { + return tableName; + } + + @JsonProperty + public String getHost() + { + return host; + } + + @JsonProperty + public String getSegment() + { + return segment; + } + + @JsonProperty + public PinotColumn getTimeColumn() + { + return timeColumn; + } + + @JsonProperty + public String getTimeFilter() + { + return timeFilter; + } + + @JsonProperty + public String getPinotFilter() + { + return pinotFilter; + } + + @JsonProperty + public long getLimit() + { + return limit; + } + + @Override + public boolean isRemotelyAccessible() + { + // only http or https is remotely accessible + return remotelyAccessible; + } + + @Override + public List getAddresses() + { + return addresses; + } + + @Override + public Object getInfo() + { + return this; + } +} diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotSplitManager.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotSplitManager.java new file mode 100644 index 0000000000000..f964b298a3f41 --- /dev/null +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotSplitManager.java @@ -0,0 +1,287 @@ +/* + * 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.ConnectorSession; +import com.facebook.presto.spi.ConnectorSplit; +import com.facebook.presto.spi.ConnectorSplitSource; +import com.facebook.presto.spi.ConnectorTableLayoutHandle; +import com.facebook.presto.spi.FixedSplitSource; +import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.block.Block; +import com.facebook.presto.spi.connector.ConnectorSplitManager; +import com.facebook.presto.spi.connector.ConnectorTransactionHandle; +import com.facebook.presto.spi.predicate.Domain; +import com.facebook.presto.spi.predicate.Marker; +import com.facebook.presto.spi.predicate.Range; +import com.facebook.presto.spi.predicate.TupleDomain; +import com.facebook.presto.spi.type.VarcharType; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Joiner; +import com.google.common.collect.ImmutableList; +import io.airlift.log.Logger; +import io.airlift.slice.Slice; + +import javax.inject.Inject; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static com.facebook.presto.pinot.PinotErrorCode.PINOT_FAILURE_GETTING_TABLE; +import static com.facebook.presto.pinot.Types.checkType; +import static com.google.common.base.Preconditions.checkState; +import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.joining; + +public class PinotSplitManager + implements ConnectorSplitManager +{ + private static final Logger log = Logger.get(PinotSplitManager.class); + + private final String connectorId; + private final PinotConfig pinotConfig; + private final PinotConnection pinotPrestoConnection; + + @Inject + public PinotSplitManager(PinotConnectorId connectorId, PinotConfig pinotConfig, PinotConnection pinotPrestoConnection) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); + this.pinotConfig = pinotConfig; + this.pinotPrestoConnection = requireNonNull(pinotPrestoConnection, "pinotPrestoConnection is null"); + } + + @Override + public ConnectorSplitSource getSplits( + ConnectorTransactionHandle transactionHandle, + ConnectorSession session, + ConnectorTableLayoutHandle layout, + SplitSchedulingContext splitSchedulingContext) + { + PinotTableLayoutHandle layoutHandle = checkType(layout, PinotTableLayoutHandle.class, "layout"); + PinotTableHandle tableHandle = layoutHandle.getTable(); + PinotTable table = null; + PinotColumn timeColumn = null; + Map>> routingTable = null; + Map timeBoundary = null; + try { + table = pinotPrestoConnection.getTable(tableHandle.getTableName()); + timeColumn = pinotPrestoConnection.getPinotTimeColumnForTable(tableHandle.getTableName()); + routingTable = pinotPrestoConnection.getRoutingTable(tableHandle.getTableName()); + timeBoundary = pinotPrestoConnection.getTimeBoundary(tableHandle.getTableName()); + // this can happen if table is removed during a query + checkState(table != null, "Table %s no longer exists", tableHandle.getTableName()); + } + catch (Exception e) { + throw new PrestoException( + PINOT_FAILURE_GETTING_TABLE, + "Failed to fetch table status for Pinot table: " + tableHandle.getTableName(), + e); + } + + List splits = new ArrayList<>(); + if (!routingTable.isEmpty()) { + setSplits(splits, timeColumn, routingTable, timeBoundary, getOfflineTableName(tableHandle.getTableName()), tableHandle.getConstraintSummary()); + setSplits(splits, timeColumn, routingTable, timeBoundary, getRealtimeTableName(tableHandle.getTableName()), tableHandle.getConstraintSummary()); + } + + Collections.shuffle(splits); + log.debug("PinotSplits is %s", Arrays.toString(splits.toArray())); + + return new FixedSplitSource(splits); + } + + private String getTimePredicate(String type, String timeColumn, String maxTimeStamp) + { + if ("OFFLINE".equalsIgnoreCase(type)) { + return String.format("%s < %s", timeColumn, maxTimeStamp); + } + if ("REALTIME".equalsIgnoreCase(type)) { + return String.format("%s >= %s", timeColumn, maxTimeStamp); + } + return null; + } + + private void setSplits(List splits, PinotColumn timeColumn, Map>> routingTable, Map timeBoundary, String tableName, TupleDomain constraintSummary) + { + String pinotFilter = getPinotPredicate(constraintSummary); + String timeFilter = ""; + long limit = -1; + if (timeBoundary.containsKey("timeColumnName") && timeBoundary.containsKey("timeColumnValue")) { + timeFilter = getTimePredicate(getTableType(tableName), timeBoundary.get("timeColumnName"), timeBoundary.get("timeColumnValue")); + } + for (String routingTableName : routingTable.keySet()) { + if (routingTableName.equalsIgnoreCase(tableName)) { + Map> hostToSegmentsMap = routingTable.get(routingTableName); + for (String host : hostToSegmentsMap.keySet()) { + for (String segment : hostToSegmentsMap.get(host)) { + splits.add(new PinotSplit(connectorId, routingTableName, host, segment, timeColumn, timeFilter, pinotFilter, limit)); + } + } + } + } + } + + /** + * Get the predicates for Pinot columns in string format, for constructing Pinot queries directly + * Note that for predicates like UDF (WHERE ROUND(fare) > 10), column comparison (WHERE colA - colB > 10, WHERE col/100 > 5), + * constraintSummary passed to Pinot will be empty, since those predicates would be in remainingExpression and not passed here. + * + * @param constraintSummary TupleDomain representing the allowed ranges for Pinot columns + * @return Predicate in Pinot Query Language for Pinot columns + */ + @VisibleForTesting + String getPinotPredicate(TupleDomain constraintSummary) + { + ImmutableList.Builder pinotFilterBuilder = ImmutableList.builder(); + + Map columnHandleDomainMap = constraintSummary.getDomains().get(); + for (ColumnHandle k : columnHandleDomainMap.keySet()) { + Domain domain = columnHandleDomainMap.get(k); + String columnPredicate = getColumnPredicate(domain, ((PinotColumnHandle) k).getColumnName()); + if (!columnPredicate.isEmpty()) { + pinotFilterBuilder.add("(" + columnPredicate + ")"); + } + } + return Joiner.on(" AND ").join(pinotFilterBuilder.build()); + } + + /** + * Get the predicates for a column in string format, for constructing Pinot queries directly + * + * @param domain TupleDomain representing the allowed ranges for a column + * @param columnName Pinot column name + * @return Predicate in Pinot Query Language for the column. Empty string would be returned if no constraints + */ + @VisibleForTesting + String getColumnPredicate(Domain domain, String columnName) + { + List discreteConstraintList = new ArrayList<>(); + List singleValueRangeConstraintList = new ArrayList<>(); + List rangeConstraintList = new ArrayList<>(); + + return domain.getValues().getValuesProcessor().transform( + ranges -> + { + for (Range range : ranges.getOrderedRanges()) { + if (range.isSingleValue()) { + singleValueRangeConstraintList.add(getMarkerValue(range.getLow())); + } + else { + StringBuilder builder = new StringBuilder(); + ImmutableList.Builder bounds = ImmutableList.builder(); + // Get low bound + String equationMark = (range.getLow().getBound() == Marker.Bound.EXACTLY) ? "= " : " "; + if (!range.getLow().isLowerUnbounded()) { + bounds.add(getMarkerValue(range.getLow()) + " <" + equationMark + columnName); + } + // Get high bound + equationMark = (range.getHigh().getBound() == Marker.Bound.EXACTLY) ? "= " : " "; + if (!range.getHigh().isUpperUnbounded()) { + bounds.add(columnName + " <" + equationMark + getMarkerValue(range.getHigh())); + } + // Use AND to combine bounds within the same range + builder.append("(" + Joiner.on(" AND ").join(bounds.build()) + ")"); + rangeConstraintList.add(builder.toString()); + } + } + // Multiple ranges on the same column are OR'ed together. + String rangeConstraint = Joiner.on(" OR ").join(rangeConstraintList); + String discreteConstraint = getDiscretePredicate(true, columnName, singleValueRangeConstraintList); + + return Stream.of(rangeConstraint, discreteConstraint) + .filter(s -> s != null && !s.isEmpty()) + .collect(joining(" OR ")); + }, + discreteValues -> + { + /** + * For some types like {@link com.facebook.presto.type.ColorType} that are not orderable, discreteValues would appear here. + * For most regular types like boolean, char, number, the discrete values would be converted to singleValues in ranges above, + * and would not appear here. So far the column types supported by Pinot all fall in that category. + */ + discreteConstraintList.addAll(discreteValues.getValues().stream().map(Object::toString).collect(Collectors.toList())); + return getDiscretePredicate(discreteValues.isWhiteList(), columnName, discreteConstraintList); + }, + allOrNone -> + { + // no-op + return ""; + }); + } + + /** + * Construct the IN predicate for discrete values + * + * @param isWhitelist true for IN predicate, false for NOT IN predicate + * @param columnName name of the column + * @param discreteConstraintList list of allowed or not allowed values + * @return Stringified clause with IN or NOT IN + */ + String getDiscretePredicate(boolean isWhitelist, String columnName, List discreteConstraintList) + { + if (discreteConstraintList.size() == 0) { + return ""; + } + else { + return columnName + (isWhitelist ? " " : " NOT ") + "IN (" + Joiner.on(',').join(discreteConstraintList) + ")"; + } + } + + /** + * Get the value for the Marker. + * + * @param marker marker in the Domain + * @return Underlying value for the block in the marker. For string, encapsulating quotes will be added. + */ + String getMarkerValue(Marker marker) + { + if (marker.getType() instanceof VarcharType) { + Block highBlock = marker.getValueBlock().get(); + Slice slice = highBlock.getSlice(0, 0, highBlock.getSliceLength(0)); + return "\"" + slice.toStringUtf8() + "\""; + } + else { + return marker.getValue().toString(); + } + } + + private String getOfflineTableName(String table) + { + return table + "_OFFLINE"; + } + + private String getRealtimeTableName(String table) + { + return table + "_REALTIME"; + } + + private String getTableType(String table) + { + if (table.endsWith("_REALTIME")) { + return "REALTIME"; + } + + if (table.endsWith("_OFFLINE")) { + return "OFFLINE"; + } + + return null; + } +} diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotTable.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotTable.java new file mode 100644 index 0000000000000..b25294e575498 --- /dev/null +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotTable.java @@ -0,0 +1,65 @@ +/* + * 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 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")); + + ImmutableList.Builder columnsMetadata = ImmutableList.builder(); + for (PinotColumn column : this.columns) { + columnsMetadata.add(new PinotColumnMetadata(column.getName(), column.getType())); + } + this.columnsMetadata = columnsMetadata.build(); + } + + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty + public List getColumns() + { + return columns; + } + + public List getColumnsMetadata() + { + return columnsMetadata; + } +} diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotTableHandle.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotTableHandle.java new file mode 100644 index 0000000000000..06aa798c4a86f --- /dev/null +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotTableHandle.java @@ -0,0 +1,105 @@ +/* + * 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.ConnectorTableHandle; +import com.facebook.presto.spi.SchemaTableName; +import com.facebook.presto.spi.predicate.TupleDomain; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Joiner; + +import java.util.Objects; + +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 TupleDomain constraintSummary; + + @JsonCreator + public PinotTableHandle( + @JsonProperty("connectorId") String connectorId, + @JsonProperty("schemaName") String schemaName, + @JsonProperty("tableName") String tableName) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null"); + this.schemaName = requireNonNull(schemaName, "schemaName is null"); + this.tableName = requireNonNull(tableName, "tableName is null"); + } + + @JsonProperty + public String getConnectorId() + { + return connectorId; + } + + @JsonProperty + public String getSchemaName() + { + return schemaName; + } + + @JsonProperty + public String getTableName() + { + return tableName; + } + + public TupleDomain getConstraintSummary() + { + return constraintSummary; + } + + public void setConstraintSummary(TupleDomain constraintSummary) + { + this.constraintSummary = constraintSummary; + } + + public SchemaTableName toSchemaTableName() + { + return new SchemaTableName(schemaName, tableName); + } + + @Override + public int hashCode() + { + return Objects.hash(connectorId, schemaName, tableName); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if ((obj == null) || (getClass() != obj.getClass())) { + return false; + } + + PinotTableHandle other = (PinotTableHandle) obj; + return Objects.equals(this.connectorId, other.connectorId) && Objects.equals(this.schemaName, other.schemaName) && Objects.equals(this.tableName, other.tableName); + } + + @Override + public String toString() + { + return Joiner.on(":").join(connectorId, schemaName, tableName); + } +} diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotTableLayoutHandle.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotTableLayoutHandle.java new file mode 100644 index 0000000000000..713f0b79d266e --- /dev/null +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotTableLayoutHandle.java @@ -0,0 +1,64 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.pinot; + +import com.facebook.presto.spi.ConnectorTableLayoutHandle; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +public class PinotTableLayoutHandle + implements ConnectorTableLayoutHandle +{ + private final PinotTableHandle table; + + @JsonCreator + public PinotTableLayoutHandle( + @JsonProperty("table") PinotTableHandle table) + { + this.table = table; + } + + @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/src/main/java/com/facebook/presto/pinot/PinotTransactionHandle.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotTransactionHandle.java new file mode 100644 index 0000000000000..5e0fe3f7b31e2 --- /dev/null +++ b/presto-pinot/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/src/main/java/com/facebook/presto/pinot/Types.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/Types.java new file mode 100644 index 0000000000000..5bb67c7be98f2 --- /dev/null +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/Types.java @@ -0,0 +1,33 @@ +/* + * 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 static com.google.common.base.Preconditions.checkArgument; +import static java.lang.String.format; + +final class Types +{ + private Types() + { + } + + public static B checkType(A value, Class target, String name) + { + if (value == null) { + throw new NullPointerException(format("%s is null", name)); + } + checkArgument(target.isInstance(value), "%s must be of type %s, not %s", name, target.getName(), value.getClass().getName()); + return target.cast(value); + } +} 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..0a21e8a372311 --- /dev/null +++ b/presto-pinot/src/main/resources/META-INF/services/com.facebook.presto.spi.Plugin @@ -0,0 +1 @@ +com.facebook.presto.pinot.PinotPlugin diff --git a/presto-pinot/src/test/java/com/facebook/presto/pinot/MetadataUtil.java b/presto-pinot/src/test/java/com/facebook/presto/pinot/MetadataUtil.java new file mode 100644 index 0000000000000..21e52898eaeb6 --- /dev/null +++ b/presto-pinot/src/test/java/com/facebook/presto/pinot/MetadataUtil.java @@ -0,0 +1,76 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.pinot; + +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 io.airlift.json.JsonCodec; +import io.airlift.json.JsonCodecFactory; +import io.airlift.json.ObjectMapperProvider; + +import java.util.List; +import java.util.Map; + +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 io.airlift.json.JsonCodec.listJsonCodec; +import static java.util.Locale.ENGLISH; + +public final class MetadataUtil +{ + public static final JsonCodec>> CATALOG_CODEC; + public static final JsonCodec TABLE_CODEC; + public static final JsonCodec COLUMN_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)); + if (type == null) { + throw new IllegalArgumentException(String.valueOf("Unknown type " + value)); + } + return type; + } + } + + 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); + } +} diff --git a/presto-pinot/src/test/java/com/facebook/presto/pinot/PinotQueryGeneratorTest.java b/presto-pinot/src/test/java/com/facebook/presto/pinot/PinotQueryGeneratorTest.java new file mode 100644 index 0000000000000..0517a80b7a353 --- /dev/null +++ b/presto-pinot/src/test/java/com/facebook/presto/pinot/PinotQueryGeneratorTest.java @@ -0,0 +1,57 @@ +/* + * 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.testing.assertions.Assert; +import org.testng.annotations.BeforeTest; +import org.testng.annotations.Test; + +import java.util.ArrayList; +import java.util.List; + +import static com.facebook.presto.pinot.PinotQueryGenerator.getPinotQuery; +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; + +public class PinotQueryGeneratorTest +{ + final List columnHandles = new ArrayList<>(); + + @BeforeTest + void init() + { + columnHandles.add(new PinotColumnHandle("pinot", "varchar", VARCHAR, 0)); + columnHandles.add(new PinotColumnHandle("pinot", "int", INTEGER, 1)); + columnHandles.add(new PinotColumnHandle("pinot", "secondsSinceEpoch", BIGINT, 2)); + columnHandles.add(new PinotColumnHandle("pinot", "boolean", BOOLEAN, 3)); + columnHandles.add(new PinotColumnHandle("pinot", "double", DOUBLE, 4)); + } + + @Test + public void testGetPinotQuerySelectAll() + { + String expectedQuery = "SELECT varchar, int, secondsSinceEpoch, boolean, double FROM table LIMIT 10"; + Assert.assertEquals(expectedQuery, getPinotQuery(new PinotConfig(), columnHandles, "", "", "table", 10)); + } + + @Test + public void testGetPinotQueryWithPredicate() + { + String expectedQuery = "SELECT varchar, int, secondsSinceEpoch, boolean, double FROM table WHERE ((int > 3)) AND ((secondsSinceEpoch > 10000)) LIMIT 10"; + Assert.assertEquals(expectedQuery, getPinotQuery(new PinotConfig(), columnHandles, "(int > 3)", "(secondsSinceEpoch > 10000)", "table", 10)); + } +} diff --git a/presto-pinot/src/test/java/com/facebook/presto/pinot/PinotSplitManagerTest.java b/presto-pinot/src/test/java/com/facebook/presto/pinot/PinotSplitManagerTest.java new file mode 100644 index 0000000000000..eeb91517ad677 --- /dev/null +++ b/presto-pinot/src/test/java/com/facebook/presto/pinot/PinotSplitManagerTest.java @@ -0,0 +1,178 @@ +/* + * 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.predicate.Domain; +import com.facebook.presto.spi.predicate.EquatableValueSet; +import com.facebook.presto.spi.predicate.Range; +import com.facebook.presto.spi.predicate.SortedRangeSet; +import com.facebook.presto.spi.predicate.TupleDomain; +import com.facebook.presto.spi.predicate.ValueSet; +import com.facebook.presto.testing.assertions.Assert; +import io.airlift.slice.Slices; +import org.testng.annotations.BeforeTest; +import org.testng.annotations.Test; + +import java.net.SocketException; +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; + +import static com.facebook.presto.spi.type.BigintType.BIGINT; +import static com.facebook.presto.spi.type.VarcharType.VARCHAR; +import static com.facebook.presto.type.ColorType.COLOR; + +public class PinotSplitManagerTest +{ + PinotSplitManager pinotSplitManager; + PinotColumnHandle columnCityId; + PinotColumnHandle columnCountryName; + PinotColumnHandle columnColor; + + @BeforeTest + void init() throws SocketException, UnknownHostException + { + pinotSplitManager = new PinotSplitManager(new PinotConnectorId(""), new PinotConfig(), new PinotConnection(new PinotClusterInfoFetcher(new PinotConfig()))); + columnCityId = new PinotColumnHandle("pinot", "city_id", BIGINT, 0); + columnCountryName = new PinotColumnHandle("pinot", "country_name", VARCHAR, 1); + columnColor = new PinotColumnHandle("pinot", "color", COLOR, 2); + } + + @Test + public void testSingleValueRanges() + { + Domain domain = com.facebook.presto.spi.predicate.Domain.multipleValues(BIGINT, new ArrayList<>(Arrays.asList(1L, 10L))); + String expectedFilter = "city_id IN (1,10)"; + + Assert.assertEquals(expectedFilter, pinotSplitManager.getColumnPredicate(domain, columnCityId.getColumnName())); + } + + @Test + public void testRangeValues() + { + Domain domain = Domain.create(ValueSet.ofRanges( + Range.greaterThan(BIGINT, 1L).intersect(Range.lessThan(BIGINT, 10L))), false); + + String expectedFilter = "(1 < city_id AND city_id < 10)"; + Assert.assertEquals(expectedFilter, pinotSplitManager.getColumnPredicate(domain, columnCityId.getColumnName())); + } + + @Test + public void testOneSideRanges() + { + Domain domain = Domain.create(ValueSet.ofRanges( + Range.lessThanOrEqual(BIGINT, 10L)), false); + + String expectedFilter = "(city_id <= 10)"; + Assert.assertEquals(expectedFilter, pinotSplitManager.getColumnPredicate(domain, columnCityId.getColumnName())); + } + + @Test + public void testMultipleRanges() + { + Domain domain = Domain.create(ValueSet.ofRanges( + Range.equal(BIGINT, 20L), + Range.greaterThan(BIGINT, 1L).intersect(Range.lessThan(BIGINT, 10L)), + Range.greaterThan(BIGINT, 12L).intersect(Range.lessThan(BIGINT, 18L))), false); + + String expectedFilter = "(1 < city_id AND city_id < 10) OR (12 < city_id AND city_id < 18) OR city_id IN (20)"; + Assert.assertEquals(expectedFilter, pinotSplitManager.getColumnPredicate(domain, columnCityId.getColumnName())); + } + + @Test + public void testMultipleColumns() + { + Domain domain1 = Domain.create(ValueSet.ofRanges( + Range.lessThan(BIGINT, 10L)), false); + + Domain domain2 = Domain.create(ValueSet.ofRanges( + Range.equal(VARCHAR, Slices.utf8Slice("cn")), + Range.equal(VARCHAR, Slices.utf8Slice("us"))), false); + + Map domainMap = new HashMap<>(); + domainMap.put(columnCityId, domain1); + domainMap.put(columnCountryName, domain2); + TupleDomain constraintSummary = TupleDomain.withColumnDomains(domainMap); + + String expectedFilter = "((city_id < 10)) AND (country_name IN (\"cn\",\"us\"))"; + Assert.assertEquals(expectedFilter, pinotSplitManager.getPinotPredicate(constraintSummary)); + } + + @Test + public void testNegativeDiscreteValues() + { + HashSet set = new HashSet<>(); + set.add(EquatableValueSet.ValueEntry.create(COLOR, 1L)); + set.add(EquatableValueSet.ValueEntry.create(COLOR, 2L)); + Domain domain1 = Domain.create(new EquatableValueSet(COLOR, false, set), false); + Map domainMap = new HashMap<>(); + domainMap.put(columnColor, domain1); + TupleDomain constraintSummary = TupleDomain.withColumnDomains(domainMap); + String expectedFilter = "(color NOT IN (1,2))"; + Assert.assertEquals(expectedFilter, pinotSplitManager.getPinotPredicate(constraintSummary)); + } + + /** + * Test NOT predicate. Note that types currently supported by Pinot are all orderable, + * so discrete values would appear as single values in the ranges + *

+ * In the test below, the original predicate is WHERE city_id NOT IN (1, 10). + * - The TupleDomain passed to Pinot is the allowed ranges, and instead of discrete values + * - So the final translated predicate would be the union of (-Infinity, 1), (1, 10), (10, Infinity) + * - It might not look as clean as the original predicate, but is still accurate + */ + @Test + public void testNotPredicateInRanges() + { + Domain domain1 = Domain.create(ValueSet.ofRanges( + Range.lessThan(BIGINT, 1L), + Range.greaterThan(BIGINT, 1L).intersect(Range.lessThan(BIGINT, 10L)), + Range.greaterThan(BIGINT, 10L)), false); + + Map domainMap = new HashMap<>(); + domainMap.put(columnCityId, domain1); + TupleDomain constraintSummary = TupleDomain.withColumnDomains(domainMap); + + String expectedFilter = "((city_id < 1) OR (1 < city_id AND city_id < 10) OR (10 < city_id))"; + Assert.assertEquals(expectedFilter, pinotSplitManager.getPinotPredicate(constraintSummary)); + } + + @Test + public void testEmptyDomain() + { + SortedRangeSet sortedRangeSet = SortedRangeSet.copyOf(BIGINT, new ArrayList<>()); + Domain domain = Domain.create(sortedRangeSet, false); + + Assert.assertEquals("", pinotSplitManager.getColumnPredicate(domain, columnCityId.getColumnName())); + } + + @Test + public void testEmptyConstraintSummary() + { + Assert.assertEquals("", pinotSplitManager.getPinotPredicate(TupleDomain.all())); + } + + @Test + public void testGetDiscretePredicate() + { + Assert.assertEquals("", pinotSplitManager.getDiscretePredicate(true, "city_id", new ArrayList<>())); + Assert.assertEquals("city_id IN (1,2)", pinotSplitManager.getDiscretePredicate(true, "city_id", new ArrayList<>(Arrays.asList("1", "2")))); + Assert.assertEquals("city_id NOT IN (1,2)", pinotSplitManager.getDiscretePredicate(false, "city_id", new ArrayList<>(Arrays.asList("1", "2")))); + Assert.assertEquals("country_name NOT IN (\"cn\",\"us\")", pinotSplitManager.getDiscretePredicate(false, "country_name", new ArrayList<>(Arrays.asList("\"cn\"", "\"us\"")))); + } +} diff --git a/presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotColumnHandle.java b/presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotColumnHandle.java new file mode 100644 index 0000000000000..087c1bb086a15 --- /dev/null +++ b/presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotColumnHandle.java @@ -0,0 +1,58 @@ +/* + * 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 io.airlift.testing.EquivalenceTester; +import org.testng.annotations.Test; + +import static com.facebook.presto.pinot.MetadataUtil.COLUMN_CODEC; +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("connectorId", "columnName", VARCHAR, 0); + + @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("connectorId", "columnName", VARCHAR, 0), + new PinotColumnHandle("connectorId", "columnName", VARCHAR, 0), + new PinotColumnHandle("connectorId", "columnName", BIGINT, 0), + new PinotColumnHandle("connectorId", "columnName", VARCHAR, 1)) + .addEquivalentGroup( + new PinotColumnHandle("connectorIdX", "columnName", VARCHAR, 0), + new PinotColumnHandle("connectorIdX", "columnName", VARCHAR, 0), + new PinotColumnHandle("connectorIdX", "columnName", BIGINT, 0), + new PinotColumnHandle("connectorIdX", "columnName", VARCHAR, 1)) + .addEquivalentGroup( + new PinotColumnHandle("connectorId", "columnNameX", VARCHAR, 0), + new PinotColumnHandle("connectorId", "columnNameX", VARCHAR, 0), + new PinotColumnHandle("connectorId", "columnNameX", BIGINT, 0), + new PinotColumnHandle("connectorId", "columnNameX", VARCHAR, 1)) + .check(); + } +} diff --git a/presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotConfig.java b/presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotConfig.java new file mode 100644 index 0000000000000..5607fff5abe0e --- /dev/null +++ b/presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotConfig.java @@ -0,0 +1,76 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.pinot; + +import com.google.common.collect.ImmutableMap; +import io.airlift.configuration.testing.ConfigAssertions; +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) + .setZookeeperUrl(null) + .setPinotCluster(null) + .setControllerUrl(null) + .setIdleTimeout(new Duration(5, TimeUnit.MINUTES)) + .setLimitAll(null) + .setMaxBacklogPerServer(null) + .setMaxConnectionsPerServer(null) + .setMinConnectionsPerServer(null) + .setCorePoolSize("50") + .setThreadPoolSize("64") + .setEstimatedSizeInBytesForNonNumericColumn(20) + .setConnectionTimeout(new Duration(1, TimeUnit.MINUTES))); + } + + @Test + public void testExplicitPropertyMappings() + { + Map properties = new ImmutableMap.Builder() + .put("zookeeper-uri", "localhost:2181") + .put("pinot-cluster", "pinot") + .put("controller-url", "localhost:12345") + .put("idle-timeout", "1h") + .put("limit-all", "2147483646") + .put("max-backlog-per-server", "15") + .put("max-connections-per-server", "10") + .put("min-connections-per-server", "1") + .put("core-pool-size", "100") + .put("thread-pool-size", "101") + .put("estimated-size-in-bytes-for-non-numeric-column", "30") + .put("connection-timeout", "8m").build(); + + PinotConfig expected = new PinotConfig() + .setZookeeperUrl("localhost:2181") + .setPinotCluster("pinot") + .setControllerUrl("localhost:12345") + .setIdleTimeout(new Duration(1, TimeUnit.HOURS)) + .setLimitAll("2147483646").setMaxBacklogPerServer("15") + .setMaxConnectionsPerServer("10") + .setMinConnectionsPerServer("1") + .setCorePoolSize("100") + .setThreadPoolSize("101") + .setEstimatedSizeInBytesForNonNumericColumn(30) + .setConnectionTimeout(new Duration(8, TimeUnit.MINUTES)); + + ConfigAssertions.assertFullMapping(properties, expected); + } +} diff --git a/presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotTableHandle.java b/presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotTableHandle.java new file mode 100644 index 0000000000000..f29c9f66abb8f --- /dev/null +++ b/presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotTableHandle.java @@ -0,0 +1,41 @@ +/* + * 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 io.airlift.json.JsonCodec; +import io.airlift.testing.EquivalenceTester; +import org.testng.annotations.Test; + +import static io.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-server/src/main/assembly/presto.xml b/presto-server/src/main/assembly/presto.xml index efd14285cc6c8..78ff664d48a27 100644 --- a/presto-server/src/main/assembly/presto.xml +++ b/presto-server/src/main/assembly/presto.xml @@ -164,5 +164,9 @@ ${project.build.directory}/dependency/presto-elasticsearch-${project.version} plugin/presto-elasticsearch + + ${project.build.directory}/dependency/presto-pinot-${project.version} + plugin/presto-pinot + From 991e636f0b0c8cfd61e32749b045c7956ba72ff8 Mon Sep 17 00:00:00 2001 From: Haibo Wang Date: Mon, 23 Sep 2019 16:13:51 -0700 Subject: [PATCH 2/2] Address comments first pass --- presto-pinot/pom.xml | 5 - .../presto/pinot/PinotClusterInfoFetcher.java | 79 +++---- .../presto/pinot/PinotColumnHandle.java | 17 +- .../facebook/presto/pinot/PinotErrorCode.java | 6 +- .../facebook/presto/pinot/PinotMetadata.java | 38 ++-- .../presto/pinot/PinotPageSource.java | 115 +++++----- .../presto/pinot/PinotPageSourceProvider.java | 20 +- .../presto/pinot/PinotQueryBuilder.java | 202 +++++++++++++++++ .../presto/pinot/PinotQueryGenerator.java | 80 ------- .../pinot/PinotScatterGatherQueryClient.java | 155 +++++++------ .../com/facebook/presto/pinot/PinotSplit.java | 16 +- .../presto/pinot/PinotSplitManager.java | 204 +++--------------- .../presto/pinot/PinotTableHandle.java | 15 +- .../pinot/{Types.java => PinotUtils.java} | 15 +- .../presto/pinot/PinotQueryGeneratorTest.java | 57 ----- .../presto/pinot/TestPinotColumnHandle.java | 25 +-- ...taUtil.java => TestPinotMetadataUtil.java} | 33 ++- .../presto/pinot/TestPinotQueryBuilder.java | 116 ++++++++++ ...erTest.java => TestPinotSplitManager.java} | 84 +------- .../presto/pinot/TestPinotTableHandle.java | 8 +- 20 files changed, 590 insertions(+), 700 deletions(-) create mode 100644 presto-pinot/src/main/java/com/facebook/presto/pinot/PinotQueryBuilder.java delete mode 100644 presto-pinot/src/main/java/com/facebook/presto/pinot/PinotQueryGenerator.java rename presto-pinot/src/main/java/com/facebook/presto/pinot/{Types.java => PinotUtils.java} (65%) delete mode 100644 presto-pinot/src/test/java/com/facebook/presto/pinot/PinotQueryGeneratorTest.java rename presto-pinot/src/test/java/com/facebook/presto/pinot/{MetadataUtil.java => TestPinotMetadataUtil.java} (88%) create mode 100644 presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotQueryBuilder.java rename presto-pinot/src/test/java/com/facebook/presto/pinot/{PinotSplitManagerTest.java => TestPinotSplitManager.java} (54%) diff --git a/presto-pinot/pom.xml b/presto-pinot/pom.xml index f09ee0d71aea9..afc0cf1619180 100644 --- a/presto-pinot/pom.xml +++ b/presto-pinot/pom.xml @@ -41,11 +41,6 @@ ${dep.fastjson.version} - - com.google.code.findbugs - jsr305 - - org.apache.pinot diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotClusterInfoFetcher.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotClusterInfoFetcher.java index c29a724abe2ba..3a8a0968b31ee 100644 --- a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotClusterInfoFetcher.java +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotClusterInfoFetcher.java @@ -17,12 +17,12 @@ import com.alibaba.fastjson.JSONObject; import com.facebook.presto.spi.PrestoException; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; import com.google.inject.Inject; import io.airlift.http.client.HttpStatus; import io.airlift.log.Logger; import org.apache.http.HttpEntity; import org.apache.http.HttpHeaders; -import org.apache.http.HttpResponse; import org.apache.http.client.ClientProtocolException; import org.apache.http.client.ResponseHandler; import org.apache.http.client.methods.HttpUriRequest; @@ -32,17 +32,16 @@ import org.apache.http.util.EntityUtils; import org.apache.pinot.client.DynamicBrokerSelector; import org.apache.pinot.common.data.Schema; -import org.apache.pinot.common.utils.NetUtil; +import java.io.Closeable; import java.io.IOException; -import java.net.SocketException; -import java.net.UnknownHostException; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Random; import static com.facebook.presto.pinot.PinotErrorCode.PINOT_FAILURE_GETTING_TABLE; +import static com.facebook.presto.pinot.PinotUtils.TIME_COLUMN_NAME; +import static com.facebook.presto.pinot.PinotUtils.TIME_COLUMN_VALUE; import static io.airlift.http.client.HttpStatus.familyForStatusCode; import static org.apache.pinot.common.config.TableNameBuilder.extractRawTableName; @@ -50,6 +49,7 @@ * This class handles fetching necessary information from the Pinot cluster, including table schema, broker host, routing table, etc */ public class PinotClusterInfoFetcher + implements Closeable { private static final String APPLICATION_JSON = "application/json"; @@ -57,29 +57,25 @@ public class PinotClusterInfoFetcher private static final String TABLE_SCHEMA_API_TEMPLATE = "http://%s/tables/%s/schema"; private static final String ROUTING_TABLE_API_TEMPLATE = "http://%s/debug/routingTable/%s"; private static final String TIME_BOUNDARY_API_TEMPLATE = "http://%s/debug/timeBoundary/%s"; + private static final CloseableHttpClient HTTP_CLIENT = HttpClients.createDefault(); private static final Logger log = Logger.get(PinotClusterInfoFetcher.class); - private static final CloseableHttpClient HTTP_CLIENT = HttpClients.createDefault(); private final String controllerUrl; private final String zookeeperServers; private DynamicBrokerSelector dynamicBrokerSelector; - private String instanceId = "Presto_pinot_master"; private ObjectMapper objectMapper = new ObjectMapper(); @Inject public PinotClusterInfoFetcher(PinotConfig pinotConfig) - throws SocketException, UnknownHostException { this(pinotConfig.getZookeeperUrl(), pinotConfig.getPinotCluster(), pinotConfig.getControllerUrl()); } public PinotClusterInfoFetcher(String zookeeperUrl, String pinotCluster, String controllerUrl) - throws SocketException, UnknownHostException { log.info("Trying to init PinotClusterInfoFetcher with Zookeeper: %s, PinotCluster %s, ControllerUrl: %s.", zookeeperUrl, pinotCluster, controllerUrl); zookeeperServers = zookeeperUrl + "/" + pinotCluster; - instanceId = instanceId + "_" + NetUtil.getHostAddress(); this.controllerUrl = controllerUrl; } @@ -92,23 +88,17 @@ public static String sendHttpGet(final String url) private static ResponseHandler getStringResponseHandler() { - return new ResponseHandler() - { - @Override - public String handleResponse(HttpResponse response) - throws ClientProtocolException, IOException - { - int status = response.getStatusLine().getStatusCode(); - HttpEntity entity = response.getEntity(); - if (familyForStatusCode(response.getStatusLine().getStatusCode()) == HttpStatus.Family.SUCCESSFUL && entity != null) { - return EntityUtils.toString(entity); - } - else { - if (entity == null) { - throw new ClientProtocolException("Entity is null"); - } - throw new ClientProtocolException("Unexpected status: " + status); + return response -> { + int status = response.getStatusLine().getStatusCode(); + HttpEntity entity = response.getEntity(); + if (familyForStatusCode(response.getStatusLine().getStatusCode()) == HttpStatus.Family.SUCCESSFUL && entity != null) { + return EntityUtils.toString(entity); + } + else { + if (entity == null) { + throw new ClientProtocolException("Entity is null"); } + throw new ClientProtocolException("Unexpected status: " + status); } }; } @@ -119,11 +109,6 @@ public void close() HTTP_CLIENT.close(); } - private String getControllerUrl() - { - return this.controllerUrl; - } - @SuppressWarnings("unchecked") public List getAllTables() throws Exception @@ -143,7 +128,6 @@ public Schema getTableSchema(String table) } public String getBrokerHost(String table) - throws Exception { if (dynamicBrokerSelector == null) { dynamicBrokerSelector = new DynamicBrokerSelector(zookeeperServers); @@ -154,12 +138,13 @@ public String getBrokerHost(String table) public Map>> getRoutingTableForTable(String tableName) throws Exception { - final Map>> routingTableMap = new HashMap<>(); - final String url = String.format(ROUTING_TABLE_API_TEMPLATE, getBrokerHost(tableName), tableName); + ImmutableMap.Builder>> routingTableMapBuilder = ImmutableMap.builder(); + String url = String.format(ROUTING_TABLE_API_TEMPLATE, getBrokerHost(tableName), tableName); + String responseBody = sendHttpGet(url); - log.debug("Trying to get routingTable for %s. url: %s", tableName, url); JSONObject resp = JSONObject.parseObject(responseBody); JSONArray routingTableSnapshots = resp.getJSONArray("routingTableSnapshot"); + for (int i = 0; i < routingTableSnapshots.size(); i++) { JSONObject snapshot = routingTableSnapshots.getJSONObject(i); String tableNameWithType = snapshot.getString("tableName"); @@ -176,11 +161,12 @@ public Map>> getRoutingTableForTable(String tab PINOT_FAILURE_GETTING_TABLE, "RoutingTable is empty for " + tableName); } + String routingTableEntries = routingTableEntriesArray.getJSONObject(new Random().nextInt(routingTableEntriesArray.size())).toJSONString(); Map> routingTable = objectMapper.readValue(routingTableEntries, Map.class); - routingTableMap.put(tableNameWithType, routingTable); + routingTableMapBuilder.put(tableNameWithType, routingTable); } - return routingTableMap; + return routingTableMapBuilder.build(); } public Map getTimeBoundaryForTable(String table) @@ -188,14 +174,19 @@ public Map getTimeBoundaryForTable(String table) { final String url = String.format(TIME_BOUNDARY_API_TEMPLATE, getBrokerHost(table), table); String responseBody = sendHttpGet(url); - JSONObject resp = JSONObject.parseObject(responseBody); - Map timeBoundary = new HashMap<>(); - if (resp.containsKey("timeColumnName")) { - timeBoundary.put("timeColumnName", resp.getString("timeColumnName")); + JSONObject response = JSONObject.parseObject(responseBody); + ImmutableMap.Builder timeBoundaryBuilder = ImmutableMap.builder(); + if (response.containsKey(TIME_COLUMN_NAME)) { + timeBoundaryBuilder.put(TIME_COLUMN_NAME, response.getString(TIME_COLUMN_NAME)); } - if (resp.containsKey("timeColumnValue")) { - timeBoundary.put("timeColumnValue", resp.getString("timeColumnValue")); + if (response.containsKey(TIME_COLUMN_VALUE)) { + timeBoundaryBuilder.put(TIME_COLUMN_VALUE, response.getString(TIME_COLUMN_VALUE)); } - return timeBoundary; + return timeBoundaryBuilder.build(); + } + + private String getControllerUrl() + { + return this.controllerUrl; } } diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotColumnHandle.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotColumnHandle.java index 4e6e4b8117942..941dfd213ac32 100644 --- a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotColumnHandle.java +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotColumnHandle.java @@ -20,7 +20,6 @@ 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; @@ -28,31 +27,21 @@ public final class PinotColumnHandle implements ColumnHandle { - private final String connectorId; private final String columnName; private final Type columnType; private final int ordinalPosition; - private Optional aggregationType; @JsonCreator public PinotColumnHandle( - @JsonProperty("connectorId") String connectorId, @JsonProperty("columnName") String columnName, @JsonProperty("columnType") Type columnType, @JsonProperty("ordinalPosition") int ordinalPosition) { - this.connectorId = requireNonNull(connectorId, "connectorId is null"); this.columnName = requireNonNull(columnName, "columnName is null"); this.columnType = requireNonNull(columnType, "columnType is null"); this.ordinalPosition = ordinalPosition; } - @JsonProperty - public String getConnectorId() - { - return connectorId; - } - @JsonProperty public String getColumnName() { @@ -84,7 +73,7 @@ public String getName() @Override public int hashCode() { - return Objects.hash(connectorId, columnName); + return Objects.hash(columnName); } @Override @@ -98,12 +87,12 @@ public boolean equals(Object obj) } PinotColumnHandle other = (PinotColumnHandle) obj; - return Objects.equals(this.connectorId, other.connectorId) && Objects.equals(this.columnName, other.columnName); + return Objects.equals(this.columnName, other.columnName); } @Override public String toString() { - return toStringHelper(this).add("connectorId", connectorId).add("columnName", columnName).add("columnType", columnType).add("ordinalPosition", ordinalPosition).toString(); + return toStringHelper(this).add("columnName", columnName).add("columnType", columnType).add("ordinalPosition", ordinalPosition).toString(); } } diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotErrorCode.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotErrorCode.java index b110193b598ae..abd3562cfc3d0 100644 --- a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotErrorCode.java +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotErrorCode.java @@ -18,6 +18,7 @@ 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 @@ -25,7 +26,8 @@ public enum PinotErrorCode PINOT_UNSUPPORTED_COLUMN_TYPE(0, EXTERNAL), PINOT_FAILURE_GETTING_TABLE(1, EXTERNAL), PINOT_FAILURE_GETTING_SCHEMA(2, EXTERNAL), - PINOT_FAILURE_QUERYING_DATA(3, EXTERNAL); + PINOT_FAILURE_QUERYING_DATA(3, EXTERNAL), + PINOT_FAILURE_INITIATING_RESOURCES(4, INTERNAL_ERROR); /** * Connectors can use error codes starting at the range 0x0100_0000 @@ -38,7 +40,7 @@ public enum PinotErrorCode PinotErrorCode(int code, ErrorType type) { - errorCode = new ErrorCode(code + 0x0505_0000, name(), type); + errorCode = new ErrorCode(code + 0x0106_0000, name(), type); } @Override diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotMetadata.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotMetadata.java index 5cf1a1f2b06b6..909bbdae2832a 100644 --- a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotMetadata.java +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotMetadata.java @@ -30,7 +30,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import io.airlift.log.Logger; import javax.inject.Inject; @@ -40,24 +39,20 @@ import java.util.Optional; import java.util.Set; +import static com.facebook.presto.pinot.PinotErrorCode.PINOT_FAILURE_GETTING_SCHEMA; import static com.facebook.presto.pinot.PinotErrorCode.PINOT_FAILURE_GETTING_TABLE; -import static com.facebook.presto.pinot.Types.checkType; -import static com.google.common.base.Preconditions.checkArgument; +import static com.facebook.presto.pinot.PinotUtils.checkType; import static java.util.Locale.ENGLISH; import static java.util.Objects.requireNonNull; public class PinotMetadata implements ConnectorMetadata { - private static final Logger log = Logger.get(PinotMetadata.class); - - private final String connectorId; private final PinotConnection pinotPrestoConnection; @Inject - public PinotMetadata(PinotConnectorId connectorId, PinotConnection pinotPrestoConnection) + public PinotMetadata(PinotConnection pinotPrestoConnection) { - this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); this.pinotPrestoConnection = requireNonNull(pinotPrestoConnection, "pinotPrestoConnection is null"); } @@ -71,13 +66,15 @@ public List listSchemaNames() { try { ImmutableList.Builder schemaNamesListBuilder = ImmutableList.builder(); - for (String table : pinotPrestoConnection.getTableNames()) { - schemaNamesListBuilder.add(table.toLowerCase(ENGLISH)); - } + pinotPrestoConnection.getTableNames().forEach(table -> + schemaNamesListBuilder.add(table.toLowerCase(ENGLISH))); return schemaNamesListBuilder.build(); } catch (Exception e) { - return ImmutableList.of(); + throw new PrestoException( + PINOT_FAILURE_GETTING_SCHEMA, + "Failed to list schema names.", + e); } } @@ -88,7 +85,9 @@ public String getPinotTableNameFromPrestoTableName(String prestoTableName) return pinotTableName; } } - return null; + throw new PrestoException( + PINOT_FAILURE_GETTING_SCHEMA, + String.format("Failed to get pinot table from presto table name: %s.", prestoTableName)); } @Override @@ -97,6 +96,7 @@ public PinotTableHandle getTableHandle(ConnectorSession session, SchemaTableName if (!listSchemaNames(session).contains(tableName.getSchemaName().toLowerCase(ENGLISH))) { return null; } + String pinotTableName; try { pinotTableName = getPinotTableNameFromPrestoTableName(tableName.getTableName()); @@ -111,7 +111,7 @@ public PinotTableHandle getTableHandle(ConnectorSession session, SchemaTableName String.format("Failed to get TableHandle for %s", tableName), e); } - return new PinotTableHandle(connectorId, tableName.getSchemaName(), pinotTableName); + return new PinotTableHandle(tableName.getSchemaName(), pinotTableName); } @Override @@ -133,7 +133,6 @@ public ConnectorTableLayout getTableLayout(ConnectorSession session, ConnectorTa public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle table) { PinotTableHandle pinotTableHandle = checkType(table, PinotTableHandle.class, "table"); - checkArgument(pinotTableHandle.getConnectorId().equals(connectorId), "tableHandle is not for this connector"); SchemaTableName tableName = new SchemaTableName(pinotTableHandle.getSchemaName(), pinotTableHandle.getTableName()); return getTableMetadata(tableName); @@ -167,7 +166,6 @@ public List listTables(ConnectorSession session, String schemaN public Map getColumnHandles(ConnectorSession session, ConnectorTableHandle tableHandle) { PinotTableHandle pinotTableHandle = checkType(tableHandle, PinotTableHandle.class, "tableHandle"); - checkArgument(pinotTableHandle.getConnectorId().equals(connectorId), "tableHandle is not for this connector"); String pinotTableName = getPinotTableNameFromPrestoTableName(pinotTableHandle.getTableName()); try { @@ -178,14 +176,16 @@ public Map getColumnHandles(ConnectorSession session, Conn ImmutableMap.Builder columnHandles = ImmutableMap.builder(); int index = 0; for (ColumnMetadata column : table.getColumnsMetadata()) { - columnHandles.put(column.getName().toLowerCase(ENGLISH), new PinotColumnHandle(connectorId, ((PinotColumnMetadata) column).getPinotName(), column.getType(), index)); + columnHandles.put(column.getName().toLowerCase(ENGLISH), new PinotColumnHandle(((PinotColumnMetadata) column).getPinotName(), column.getType(), index)); index++; } return columnHandles.build(); } catch (Exception e) { - log.error("Failed to get ColumnHandles for table : " + pinotTableHandle.getTableName(), e); - return null; + throw new PrestoException( + PINOT_FAILURE_GETTING_TABLE, + String.format("Failed to get ColumnHandles for table : %s.", pinotTableHandle.getTableName()), + e); } } diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotPageSource.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotPageSource.java index 9d41461cfe2af..2ff49aaf6f20c 100644 --- a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotPageSource.java +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotPageSource.java @@ -29,6 +29,7 @@ import org.apache.pinot.common.utils.DataTable; import java.util.Arrays; +import java.util.Deque; import java.util.HashMap; import java.util.LinkedList; import java.util.List; @@ -36,7 +37,7 @@ import java.util.stream.IntStream; import static com.facebook.presto.pinot.PinotErrorCode.PINOT_UNSUPPORTED_COLUMN_TYPE; -import static com.facebook.presto.pinot.PinotQueryGenerator.getPinotQuery; +import static com.facebook.presto.pinot.PinotQueryBuilder.getPinotQuery; 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; @@ -45,7 +46,7 @@ import static java.util.Objects.requireNonNull; /** - * This class retrieves Pinot data from a Pinot client, and re-constructs the data into Presto Pages. + * This class retrieves Pinot data from a Pinot client and re-constructs the data into Presto Pages. */ public class PinotPageSource @@ -53,15 +54,15 @@ public class PinotPageSource { private static final Logger log = Logger.get(PinotPageSource.class); - private List columnHandles; - private List columnTypes; + private static List columnHandles; + private static List columnTypes; private PinotConfig pinotConfig; private PinotSplit split; private PinotScatterGatherQueryClient pinotQueryClient; // 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 Deque dataTableList = new LinkedList<>(); private long completedBytes; private long readTimeNanos; private long estimatedMemoryUsageInBytes; @@ -141,8 +142,7 @@ public Page getNextPage() Type columnType = columnTypes.get(columnHandleIdx); writeBlock(blockBuilder, columnType, pinotColumnNameIndexMap.get(columnHandles.get(columnHandleIdx).getColumnName())); } - Page page = pageBuilder.build(); - return page; + return pageBuilder.build(); } /** @@ -152,9 +152,11 @@ private void fetchPinotData() { log.debug("Fetching data from Pinot for table %s, segment %s", split.getTableName(), split.getSegment()); long startTimeNanos = System.nanoTime(); - int idx = 0; - for (PinotColumnHandle columnHandle : columnHandles) { - pinotColumnNameIndexMap.put(columnHandle.getColumnName(), idx++); + if (pinotColumnNameIndexMap.isEmpty()) { + int idx = 0; + for (PinotColumnHandle columnHandle : columnHandles) { + pinotColumnNameIndexMap.put(columnHandle.getColumnName(), idx++); + } } String pinotQuery = getPinotQuery(pinotConfig, columnHandles, split.getPinotFilter(), split.getTimeFilter(), split.getTableName(), split.getLimit()); Map dataTableMap = pinotQueryClient.queryPinotServerForDataTable(pinotQuery, split.getHost(), split.getSegment()); @@ -172,12 +174,11 @@ private void fetchPinotData() estimatedMemoryUsageInBytes += estimatedTableSizeInBytes; }); ImmutableList.Builder types = ImmutableList.builder(); - columnHandles - .stream() - .map(columnHandle -> getTypeForBlock(columnHandle)) + columnHandles.stream() + .map(PinotPageSource::getTypeForBlock) .forEach(types::add); - this.columnTypes = types.build(); - readTimeNanos = System.nanoTime() - startTimeNanos; + columnTypes = types.build(); + readTimeNanos += System.nanoTime() - startTimeNanos; isPinotDataFetched = true; } @@ -263,71 +264,67 @@ private void writeSliceBlock(BlockBuilder blockBuilder, Type columnType, int col }); } - Type getType(int colIdx) + private static Type getType(int columnIndex) { - checkArgument(colIdx < columnHandles.size(), "Invalid field index"); - return columnHandles.get(colIdx).getColumnType(); + checkArgument(columnIndex < columnHandles.size(), "Invalid field index"); + return columnHandles.get(columnIndex).getColumnType(); } - boolean getBoolean(int rowIdx, int colIdx) + private boolean getBoolean(int rowIndex, int columnIndex) { - return Boolean.getBoolean(currentDataTable.getDataTable().getString(rowIdx, colIdx)); + return Boolean.getBoolean(currentDataTable.getDataTable().getString(rowIndex, columnIndex)); } - long getLong(int rowIdx, int colIdx) + private long getLong(int rowIndex, int columnIndex) { - DataSchema.ColumnDataType dataType = currentDataTable.getDataTable().getDataSchema().getColumnDataType(colIdx); + 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(rowIdx, colIdx); - } - if (dataType.equals(DataType.INT)) { - return (long) currentDataTable.getDataTable().getInt(rowIdx, colIdx); + if (dataType.equals(DataSchema.ColumnDataType.DOUBLE)) { + return (long) currentDataTable.getDataTable().getDouble(rowIndex, columnIndex); } - else { - return currentDataTable.getDataTable().getLong(rowIdx, colIdx); + if (dataType.equals(DataSchema.ColumnDataType.INT)) { + return (long) currentDataTable.getDataTable().getInt(rowIndex, columnIndex); } + return currentDataTable.getDataTable().getLong(rowIndex, columnIndex); } - double getDouble(int rowIdx, int colIdx) + private double getDouble(int rowIndex, int columnIndex) { - DataSchema.ColumnDataType dataType = currentDataTable.getDataTable().getDataSchema().getColumnDataType(colIdx); + DataSchema.ColumnDataType dataType = currentDataTable.getDataTable().getDataSchema().getColumnDataType(columnIndex); if (dataType.equals(DataType.FLOAT)) { - return currentDataTable.getDataTable().getFloat(rowIdx, colIdx); - } - else { - return currentDataTable.getDataTable().getDouble(rowIdx, colIdx); + return currentDataTable.getDataTable().getFloat(rowIndex, columnIndex); } + return currentDataTable.getDataTable().getDouble(rowIndex, columnIndex); } - Slice getSlice(int rowIdx, int colIdx) + private Slice getSlice(int rowIndex, int columnIndex) { - checkColumnType(colIdx, VARCHAR); - DataSchema.ColumnDataType columnType = currentDataTable.getDataTable().getDataSchema().getColumnDataType(colIdx); + checkColumnType(columnIndex, VARCHAR); + DataSchema.ColumnDataType columnType = currentDataTable.getDataTable().getDataSchema().getColumnDataType(columnIndex); switch (columnType) { case INT_ARRAY: - int[] intArray = currentDataTable.getDataTable().getIntArray(rowIdx, colIdx); + int[] intArray = currentDataTable.getDataTable().getIntArray(rowIndex, columnIndex); return utf8Slice(Arrays.toString(intArray)); case LONG_ARRAY: - long[] longArray = currentDataTable.getDataTable().getLongArray(rowIdx, colIdx); + long[] longArray = currentDataTable.getDataTable().getLongArray(rowIndex, columnIndex); return utf8Slice(Arrays.toString(longArray)); case FLOAT_ARRAY: - float[] floatArray = currentDataTable.getDataTable().getFloatArray(rowIdx, colIdx); + float[] floatArray = currentDataTable.getDataTable().getFloatArray(rowIndex, columnIndex); return utf8Slice(Arrays.toString(floatArray)); case DOUBLE_ARRAY: - double[] doubleArray = currentDataTable.getDataTable().getDoubleArray(rowIdx, colIdx); + double[] doubleArray = currentDataTable.getDataTable().getDoubleArray(rowIndex, columnIndex); return utf8Slice(Arrays.toString(doubleArray)); case STRING_ARRAY: - String[] stringArray = currentDataTable.getDataTable().getStringArray(rowIdx, colIdx); + String[] stringArray = currentDataTable.getDataTable().getStringArray(rowIndex, columnIndex); return utf8Slice(Arrays.toString(stringArray)); case STRING: - String fieldStr = currentDataTable.getDataTable().getString(rowIdx, colIdx); - if (fieldStr == null || fieldStr.isEmpty()) { + String field = currentDataTable.getDataTable().getString(rowIndex, columnIndex); + if (field == null || field.isEmpty()) { return Slices.EMPTY_SLICE; } - return Slices.utf8Slice(fieldStr); + return Slices.utf8Slice(field); } return Slices.EMPTY_SLICE; } @@ -354,44 +351,40 @@ private int getEstimatedColumnSizeInBytes(DataSchema.ColumnDataType dataType) return Integer.BYTES; } } - else { - return pinotConfig.getEstimatedSizeInBytesForNonNumericColumn(); - } + return pinotConfig.getEstimatedSizeInBytesForNonNumericColumn(); } - void checkColumnType(int colIdx, Type expected) + private static void checkColumnType(int columnIndex, Type expected) { - Type actual = getType(colIdx); - checkArgument(actual.equals(expected), "Expected column %s to be type %s but is %s", colIdx, expected, actual); + 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) + private static Type getTypeForBlock(PinotColumnHandle pinotColumnHandle) { if (pinotColumnHandle.getColumnType().equals(INTEGER)) { return BIGINT; } - else { - return pinotColumnHandle.getColumnType(); - } + return pinotColumnHandle.getColumnType(); } - private class PinotDataTableWithSize + private static class PinotDataTableWithSize { - DataTable dataTable; + final DataTable dataTable; int estimatedSizeInBytes; PinotDataTableWithSize(DataTable dataTable, int estimatedSizeInBytes) { - this.dataTable = dataTable; + this.dataTable = requireNonNull(dataTable); this.estimatedSizeInBytes = estimatedSizeInBytes; } - DataTable getDataTable() + public DataTable getDataTable() { return dataTable; } - int getEstimatedSizeInBytes() + public int getEstimatedSizeInBytes() { return estimatedSizeInBytes; } diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotPageSourceProvider.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotPageSourceProvider.java index dda39132d9dd7..a25b120e78e93 100644 --- a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotPageSourceProvider.java +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotPageSourceProvider.java @@ -19,27 +19,24 @@ import com.facebook.presto.spi.ConnectorSplit; import com.facebook.presto.spi.connector.ConnectorPageSourceProvider; import com.facebook.presto.spi.connector.ConnectorTransactionHandle; +import com.google.common.collect.ImmutableList; import javax.inject.Inject; -import java.util.ArrayList; import java.util.List; -import static com.facebook.presto.pinot.Types.checkType; -import static com.google.common.base.Preconditions.checkArgument; +import static com.facebook.presto.pinot.PinotUtils.checkType; import static java.util.Objects.requireNonNull; public class PinotPageSourceProvider implements ConnectorPageSourceProvider { - private final String connectorId; private final PinotConfig pinotConfig; private final PinotScatterGatherQueryClient pinotQueryClient; @Inject - public PinotPageSourceProvider(PinotConnectorId connectorId, PinotConfig pinotConfig, PinotScatterGatherQueryClient pinotQueryClient) + public PinotPageSourceProvider(PinotConfig pinotConfig, PinotScatterGatherQueryClient pinotQueryClient) { - this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); this.pinotConfig = requireNonNull(pinotConfig, "pinotConfig is null"); this.pinotQueryClient = requireNonNull(pinotQueryClient, "pinotQueryClient is null"); } @@ -49,19 +46,16 @@ public ConnectorPageSource createPageSource(ConnectorTransactionHandle transacti { requireNonNull(split, "partitionChunk is null"); PinotSplit pinotSplit = checkType(split, PinotSplit.class, "split"); - checkArgument(pinotSplit.getConnectorId().equals(connectorId), "split is not for this connector"); - List handles = new ArrayList<>(); + ImmutableList.Builder handlesBuilder = ImmutableList.builder(); if (columns.isEmpty()) { // For COUNT(*) and COUNT(1), no columns are passed down to Pinot // Since this is the only known type of queries for this scenario, we just select time column from Pinot to facilitate the COUNT - handles.add(new PinotColumnHandle(this.connectorId, pinotSplit.getTimeColumn().getName(), pinotSplit.getTimeColumn().getType(), 0)); + handlesBuilder.add(new PinotColumnHandle(pinotSplit.getTimeColumn().getName(), pinotSplit.getTimeColumn().getType(), 0)); } else { - for (ColumnHandle handle : columns) { - handles.add(checkType(handle, PinotColumnHandle.class, "handle")); - } + columns.forEach(handle -> handlesBuilder.add(checkType(handle, PinotColumnHandle.class, "handle"))); } - return new PinotPageSource(this.pinotConfig, this.pinotQueryClient, pinotSplit, handles); + return new PinotPageSource(pinotConfig, pinotQueryClient, pinotSplit, handlesBuilder.build()); } } diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotQueryBuilder.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotQueryBuilder.java new file mode 100644 index 0000000000000..e40cd68d14603 --- /dev/null +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotQueryBuilder.java @@ -0,0 +1,202 @@ +/* + * 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.Block; +import com.facebook.presto.spi.predicate.Domain; +import com.facebook.presto.spi.predicate.Marker; +import com.facebook.presto.spi.predicate.Range; +import com.facebook.presto.spi.type.VarcharType; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Joiner; +import com.google.common.collect.ImmutableList; +import io.airlift.log.Logger; +import io.airlift.slice.Slice; +import org.apache.pinot.common.utils.CommonConstants; + +import java.util.ArrayList; +import java.util.List; +import java.util.StringJoiner; +import java.util.stream.Stream; + +import static com.facebook.presto.pinot.PinotUtils.QUOTE; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.joining; + +/** + * This class manages how to generate the query to send to Pinot servers. + */ +public final class PinotQueryBuilder +{ + private static final Logger log = Logger.get(PinotQueryBuilder.class); + + private PinotQueryBuilder() + { + } + + /** + * QUERY_TEMPLATE looks like this: + * SELECT $fields FROM $tableName $predicate LIMIT $limit. + *

+ * Note $predicate is optional, and we intentionally add a space between $tableName $predicate for readability. + * When $predicate is absent, there would be 2 spaces between $tableName and LIMIT, which is should not hurt the query itself. + */ + private static final String QUERY_TEMPLATE = "SELECT %s FROM %s %s LIMIT %d"; + + /** + * Returns the Pinot Query to send for each split. + * + *

Pinot Query would be constructed based on {$link #QUERY_TEMPLATE} and predicates (WHERE ...). + * + * @return the constructed Pinot Query + */ + static String getPinotQuery(PinotConfig pinotConfig, List columnHandles, String pinotFilter, String timeFilter, String tableName, long splitLimit) + { + requireNonNull(pinotConfig, "pinotConfig is null"); + StringJoiner fieldsJoiner = new StringJoiner(", "); + for (PinotColumnHandle columnHandle : columnHandles) { + // No aggregation pushdown + fieldsJoiner.add(columnHandle.getColumnName()); + } + + // Add predicates + StringJoiner predicatesJoiner = new StringJoiner(" AND "); + if (!pinotFilter.isEmpty()) { + predicatesJoiner.add(String.format("(%s)", pinotFilter)); + } + if (!timeFilter.isEmpty()) { + predicatesJoiner.add(String.format("(%s)", timeFilter)); + } + + // Note pinotPredicate is optional. It would be empty when no predicates are pushed down. + String pinotPredicate = ""; + if (predicatesJoiner.length() > 0) { + pinotPredicate = "WHERE " + predicatesJoiner.toString(); + } + + long limit = splitLimit > 0 ? splitLimit : pinotConfig.getLimitAll(); + + String finalQuery = String.format(QUERY_TEMPLATE, fieldsJoiner.toString(), tableName, pinotPredicate, limit); + log.debug("Plan to send PQL : %s", finalQuery); + return finalQuery; + } + + /** + * Get the predicates for a column in string format, for constructing Pinot queries directly + * + * @param domain TupleDomain representing the allowed ranges for a column + * @param columnName Pinot column name + * @return Predicate in Pinot Query Language for the column. Empty string would be returned if no constraints + */ + @VisibleForTesting + static String getColumnPredicate(Domain domain, String columnName) + { + List discreteConstraintList = new ArrayList<>(); + List singleValueRangeConstraintList = new ArrayList<>(); + List rangeConstraintList = new ArrayList<>(); + + return domain.getValues().getValuesProcessor().transform( + ranges -> + { + for (Range range : ranges.getOrderedRanges()) { + if (range.isSingleValue()) { + singleValueRangeConstraintList.add(getMarkerValue(range.getLow())); + } + else { + StringBuilder builder = new StringBuilder(); + ImmutableList.Builder bounds = ImmutableList.builder(); + // Get low bound + String equationMark = (range.getLow().getBound() == Marker.Bound.EXACTLY) ? "= " : " "; + if (!range.getLow().isLowerUnbounded()) { + bounds.add(getMarkerValue(range.getLow()) + " <" + equationMark + columnName); + } + // Get high bound + equationMark = (range.getHigh().getBound() == Marker.Bound.EXACTLY) ? "= " : " "; + if (!range.getHigh().isUpperUnbounded()) { + bounds.add(columnName + " <" + equationMark + getMarkerValue(range.getHigh())); + } + // Use AND to combine bounds within the same range + builder.append("(").append(Joiner.on(" AND ").join(bounds.build())).append(")"); + rangeConstraintList.add(builder.toString()); + } + } + // Multiple ranges on the same column are OR'ed together. + String rangeConstraint = Joiner.on(" OR ").join(rangeConstraintList); + String discreteConstraint = getDiscretePredicate(true, columnName, singleValueRangeConstraintList); + + return Stream.of(rangeConstraint, discreteConstraint) + .filter(s -> !s.isEmpty()) + .collect(joining(" OR ")); + }, + discreteValues -> + { + /* + * For some types like {@link com.facebook.presto.type.ColorType} that are not orderable, discreteValues would appear here. + * For most regular types like boolean, char, number, the discrete values would be converted to singleValues in ranges above, + * and would not appear here. So far the column types supported by Pinot all fall in that category. + */ + discreteConstraintList.addAll(discreteValues.getValues().stream().map(Object::toString).collect(toImmutableList())); + return getDiscretePredicate(discreteValues.isWhiteList(), columnName, discreteConstraintList); + }, + allOrNone -> + { + // no-op + return ""; + }); + } + + /** + * Construct the IN predicate for discrete values + * + * @param isWhitelist true for IN predicate, false for NOT IN predicate + * @param columnName name of the column + * @param discreteConstraintList list of allowed or not allowed values + * @return Stringified clause with IN or NOT IN + */ + static String getDiscretePredicate(boolean isWhitelist, String columnName, List discreteConstraintList) + { + if (discreteConstraintList.size() == 0) { + return ""; + } + return columnName + (isWhitelist ? " " : " NOT ") + "IN (" + Joiner.on(',').join(discreteConstraintList) + ")"; + } + + /** + * Get the value for the Marker. + * + * @param marker marker in the Domain + * @return Underlying value for the block in the marker. For string, encapsulating quotes will be added. + */ + private static String getMarkerValue(Marker marker) + { + if (marker.getType() instanceof VarcharType) { + Block highBlock = marker.getValueBlock().get(); + Slice slice = highBlock.getSlice(0, 0, highBlock.getSliceLength(0)); + return QUOTE + slice.toStringUtf8() + QUOTE; + } + return marker.getValue().toString(); + } + + static String getTimePredicate(String tableType, String timeColumn, String maxTimeStamp) + { + if (CommonConstants.Helix.TableType.OFFLINE.toString().equalsIgnoreCase(tableType)) { + return String.format("%s < %s", timeColumn, maxTimeStamp); + } + if (CommonConstants.Helix.TableType.REALTIME.toString().equalsIgnoreCase(tableType)) { + return String.format("%s >= %s", timeColumn, maxTimeStamp); + } + return null; + } +} diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotQueryGenerator.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotQueryGenerator.java deleted file mode 100644 index 8bbaf5aa34149..0000000000000 --- a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotQueryGenerator.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.facebook.presto.pinot; - -import io.airlift.log.Logger; - -import java.util.List; -import java.util.StringJoiner; - -import static java.util.Objects.requireNonNull; - -/** - * This class manages how to generate the query to send to Pinot servers. - */ -public final class PinotQueryGenerator -{ - private static final Logger log = Logger.get(PinotQueryGenerator.class); - - private PinotQueryGenerator() - { - } - - /** - * QUERY_TEMPLATE looks like this: - * SELECT $fields FROM $tableName $predicate LIMIT $limit. - *

- * Note $predicate is optional, and we intentionally add a space between $tableName $predicate for readability. - * When $predicate is absent, there would be 2 spaces between $tableName and LIMIT, which is should not hurt the query itself. - */ - public static final String QUERY_TEMPLATE = "SELECT %s FROM %s %s LIMIT %d"; - - /** - * Returns the Pinot Query to send for each split. - * - *

Pinot Query would be constructed based on {$link #QUERY_TEMPLATE} and predicates (WHERE ...). - * - * @return the constructed Pinot Query - */ - static String getPinotQuery(PinotConfig pinotConfig, List columnHandles, String pinotFilter, String timeFilter, String tableName, long splitLimit) - { - requireNonNull(pinotConfig, "pinotConfig is null"); - StringJoiner fieldsJoiner = new StringJoiner(", "); - for (PinotColumnHandle columnHandle : columnHandles) { - // No aggregation pushdown - fieldsJoiner.add(columnHandle.getColumnName()); - } - - // Add predicates - StringJoiner predicatesJoiner = new StringJoiner(" AND "); - if (!pinotFilter.isEmpty()) { - predicatesJoiner.add(String.format("(%s)", pinotFilter)); - } - if (!timeFilter.isEmpty()) { - predicatesJoiner.add(String.format("(%s)", timeFilter)); - } - - // Note pinotPredicate is optional. It would be empty when no predicates are pushed down. - String pinotPredicate = ""; - if (predicatesJoiner.length() > 0) { - pinotPredicate = "WHERE " + predicatesJoiner.toString(); - } - - long limit = splitLimit > 0 ? splitLimit : pinotConfig.getLimitAll(); - - final String finalQuery = String.format(QUERY_TEMPLATE, fieldsJoiner.toString(), tableName, pinotPredicate, limit); - log.debug("Plan to send PQL : %s", finalQuery); - return finalQuery; - } -} diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotScatterGatherQueryClient.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotScatterGatherQueryClient.java index 84dc9dc174c2c..3117cff46075e 100644 --- a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotScatterGatherQueryClient.java +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotScatterGatherQueryClient.java @@ -14,6 +14,8 @@ package com.facebook.presto.pinot; import com.facebook.presto.spi.PrestoException; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.inject.Inject; import com.yammer.metrics.core.MetricsRegistry; import io.airlift.log.Logger; @@ -43,14 +45,9 @@ import org.apache.pinot.transport.scattergather.ScatterGatherStats; import org.apache.thrift.protocol.TCompactProtocol; -import javax.annotation.Nonnull; -import javax.annotation.Nullable; - import java.net.InetAddress; import java.net.UnknownHostException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.concurrent.ExecutorService; @@ -59,6 +56,8 @@ import java.util.concurrent.atomic.AtomicLong; import static com.facebook.presto.pinot.PinotErrorCode.PINOT_FAILURE_QUERYING_DATA; +import static com.google.common.base.Preconditions.checkState; +import static java.util.Objects.requireNonNull; /** * This class acts as the Pinot broker, fetches data from Pinot segments, gathers and returns the result. @@ -74,16 +73,12 @@ public class PinotScatterGatherQueryClient private final AtomicLong requestIdGenerator; private final String prestoHostId; - private final MetricsRegistry registry; private final BrokerMetrics brokerMetrics; private final ScatterGather scatterGatherer; + // Netty Specific private EventLoopGroup eventLoopGroup; - private PooledNettyClientResourceManager resourceManager; - // Connection Pool Related - private KeyedPool connPool; - private ScheduledThreadPoolExecutor poolTimeoutExecutor; - private ExecutorService requestSenderPool; + private Duration connectionTimeout; @Inject @@ -92,12 +87,12 @@ public PinotScatterGatherQueryClient(PinotConfig pinotConfig) requestIdGenerator = new AtomicLong(0); prestoHostId = getDefaultPrestoId(); - registry = new MetricsRegistry(); + final MetricsRegistry registry = new MetricsRegistry(); brokerMetrics = new BrokerMetrics(registry, DEFAULT_EMIT_TABLE_LEVEL_METRICS); brokerMetrics.initializeGlobalMeters(); - eventLoopGroup = new NioEventLoopGroup(); - /** + + /* * Some of the client metrics uses histogram which is doing synchronous operation. * These are fixed overhead per request/response. * TODO: Measure the overhead of this. @@ -105,16 +100,24 @@ public PinotScatterGatherQueryClient(PinotConfig pinotConfig) final NettyClientMetrics clientMetrics = new NettyClientMetrics(registry, "presto_pinot_client_"); // Setup Netty Connection Pool - resourceManager = new PooledNettyClientResourceManager(eventLoopGroup, new HashedWheelTimer(), clientMetrics); - - requestSenderPool = Executors.newFixedThreadPool(pinotConfig.getThreadPoolSize()); - poolTimeoutExecutor = new ScheduledThreadPoolExecutor(pinotConfig.getCorePoolSize()); + PooledNettyClientResourceManager resourceManager = new PooledNettyClientResourceManager(eventLoopGroup, new HashedWheelTimer(), clientMetrics); + // Connection Pool Related + ExecutorService requestSenderPool = Executors.newFixedThreadPool(pinotConfig.getThreadPoolSize()); + ScheduledThreadPoolExecutor poolTimeoutExecutor = new ScheduledThreadPoolExecutor(pinotConfig.getCorePoolSize()); connectionTimeout = pinotConfig.getConnectionTimeout(); - connPool = new KeyedPoolImpl(pinotConfig.getMinConnectionsPerServer(), pinotConfig.getMaxConnectionsPerServer(), pinotConfig.getIdleTimeout().toMillis(), pinotConfig.getMaxBacklogPerServer(), resourceManager, poolTimeoutExecutor, requestSenderPool, registry); - resourceManager.setPool(connPool); + KeyedPool connectionPool = new KeyedPoolImpl<>( + pinotConfig.getMinConnectionsPerServer(), + pinotConfig.getMaxConnectionsPerServer(), + pinotConfig.getIdleTimeout().toMillis(), + pinotConfig.getMaxBacklogPerServer(), + resourceManager, + poolTimeoutExecutor, + requestSenderPool, + registry); + resourceManager.setPool(connectionPool); // Setup ScatterGather - scatterGatherer = new ScatterGatherImpl(connPool, requestSenderPool); + scatterGatherer = new ScatterGatherImpl(connectionPool, requestSenderPool); } private String getDefaultPrestoId() @@ -144,27 +147,26 @@ public Map queryPinotServerForDataTable(String pql, S e); } - Map> routingTable = new HashMap<>(); - List segmentList = new ArrayList<>(); - segmentList.add(segment); - routingTable.put(serverHost, segmentList); - ScatterGatherRequestImpl scatterRequest = new ScatterGatherRequestImpl(brokerRequest, routingTable, 0, connectionTimeout.toMillis(), prestoHostId); + ImmutableMap.Builder> routingTableBuilder = ImmutableMap.builder(); + List segmentList = Arrays.asList(segment); + routingTableBuilder.put(serverHost, segmentList); + ScatterGatherRequest scatterRequest = new SimpleScatterGatherRequest(brokerRequest, routingTableBuilder.build(), 0, connectionTimeout.toMillis(), prestoHostId); ScatterGatherStats scatterGatherStats = new ScatterGatherStats(); CompositeFuture compositeFuture = routeScatterGather(scatterRequest, scatterGatherStats); if (compositeFuture == null) { - // No server found in either OFFLINE or REALTIME table. - return null; + throw new PrestoException( + PINOT_FAILURE_QUERYING_DATA, + String.format("Failed to get data from table. PQL = %s.", pql)); } - Map dataTableMap = new HashMap<>(); + ImmutableMap.Builder dataTableMapBuilder = ImmutableMap.builder(); + ImmutableList.Builder processingExceptionsBuilder = ImmutableList.builder(); + Map serverResponseMap = gatherServerResponses(compositeFuture, scatterGatherStats, true, brokerRequest.getQuerySource().getTableName(), processingExceptionsBuilder); - List processingExceptions = new ArrayList<>(); - Map serverResponseMap = null; - serverResponseMap = gatherServerResponses(compositeFuture, scatterGatherStats, true, brokerRequest.getQuerySource().getTableName(), processingExceptions); - deserializeServerResponses(serverResponseMap, true, dataTableMap, brokerRequest.getQuerySource().getTableName(), processingExceptions); - return dataTableMap; + deserializeServerResponses(serverResponseMap, dataTableMapBuilder, brokerRequest.getQuerySource().getTableName(), processingExceptionsBuilder); + return dataTableMapBuilder.build(); } /** @@ -174,26 +176,20 @@ public Map queryPinotServerForDataTable(String pql, S * @param scatterGatherStats scatter-gather statistics. * @param isOfflineTable whether the scatter-gather target is an OFFLINE table. * @param tableNameWithType table name with type suffix. - * @param processingExceptions list of processing exceptions. + * @param processingExceptionsBuilder list of processing exceptions. * @return server response map. */ - @Nullable private Map gatherServerResponses( - @Nonnull CompositeFuture compositeFuture, - @Nonnull ScatterGatherStats scatterGatherStats, boolean isOfflineTable, - @Nonnull String tableNameWithType, - @Nonnull List processingExceptions) + CompositeFuture compositeFuture, + ScatterGatherStats scatterGatherStats, + boolean isOfflineTable, + String tableNameWithType, + ImmutableList.Builder processingExceptionsBuilder) { try { Map serverResponseMap = compositeFuture.get(); - Iterator> iterator = serverResponseMap.entrySet().iterator(); - while (iterator.hasNext()) { - Map.Entry entry = iterator.next(); - if (entry.getValue().length == 0) { - throw new PrestoException( - PINOT_FAILURE_QUERYING_DATA, - String.format("Got empty data for table: %s in server %s.", tableNameWithType, entry.getKey().getShortHostName())); - } + for (Map.Entry entry : serverResponseMap.entrySet()) { + checkState(entry.getValue().length > 0, "Got empty data for table: %s in server %s.", tableNameWithType, entry.getKey().getShortHostName()); } Map responseTimes = compositeFuture.getResponseTimes(); scatterGatherStats.setResponseTimeMillis(responseTimes, isOfflineTable); @@ -201,10 +197,10 @@ private Map gatherServerResponses( } catch (Exception e) { brokerMetrics.addMeteredTableValue(tableNameWithType, BrokerMeter.RESPONSE_FETCH_EXCEPTIONS, 1L); - processingExceptions.add(QueryException.getException(QueryException.BROKER_GATHER_ERROR, e)); + processingExceptionsBuilder.add(QueryException.getException(QueryException.BROKER_GATHER_ERROR, e)); throw new PrestoException( PINOT_FAILURE_QUERYING_DATA, - String.format("Caught exception while fetching responses for table: %s", tableNameWithType), + String.format("Caught exception while fetching responses for table: %s. Processing Exceptions: %s", tableNameWithType, processingExceptionsBuilder.build().toString()), e); } } @@ -216,28 +212,24 @@ private Map gatherServerResponses( * them. * * @param responseMap map from server to response. - * @param isOfflineTable whether the responses are from an OFFLINE table. - * @param dataTableMap map from server to data table. + * @param dataTableMapBuilder map from server to data table. * @param tableNameWithType table name with type suffix. - * @param processingExceptions list of processing exceptions. + * @param processingExceptionsBuilder list of processing exceptions. */ private void deserializeServerResponses( - @Nonnull Map responseMap, boolean isOfflineTable, - @Nonnull Map dataTableMap, - @Nonnull String tableNameWithType, - @Nonnull List processingExceptions) + Map responseMap, + ImmutableMap.Builder dataTableMapBuilder, + String tableNameWithType, + ImmutableList.Builder processingExceptionsBuilder) { for (Map.Entry entry : responseMap.entrySet()) { ServerInstance serverInstance = entry.getKey(); - if (!isOfflineTable) { - serverInstance = serverInstance.withSeq(1); - } try { - dataTableMap.put(serverInstance, DataTableFactory.getDataTable(entry.getValue())); + dataTableMapBuilder.put(serverInstance, DataTableFactory.getDataTable(entry.getValue())); } catch (Exception e) { brokerMetrics.addMeteredTableValue(tableNameWithType, BrokerMeter.DATA_TABLE_DESERIALIZATION_EXCEPTIONS, 1L); - processingExceptions.add(QueryException.getException(QueryException.DATA_TABLE_DESERIALIZATION_ERROR, e)); + processingExceptionsBuilder.add(QueryException.getException(QueryException.DATA_TABLE_DESERIALIZATION_ERROR, e)); throw new PrestoException( PINOT_FAILURE_QUERYING_DATA, String.format("Caught exceptions while deserializing response for table: %s from server: %s", tableNameWithType, serverInstance), @@ -246,22 +238,20 @@ private void deserializeServerResponses( } } - private CompositeFuture routeScatterGather(ScatterGatherRequestImpl scatterRequest, ScatterGatherStats scatterGatherStats) + private CompositeFuture routeScatterGather(ScatterGatherRequest scatterRequest, ScatterGatherStats scatterGatherStats) { - CompositeFuture compositeFuture = null; try { - compositeFuture = this.scatterGatherer.scatterGather(scatterRequest, scatterGatherStats, true, brokerMetrics); + return scatterGatherer.scatterGather(scatterRequest, scatterGatherStats, true, brokerMetrics); } catch (InterruptedException e) { throw new PrestoException( PINOT_FAILURE_QUERYING_DATA, - "Caught exception querying Pinot servers.", + "Caught exception querying Pinot servers", e); } - return compositeFuture; } - private static class ScatterGatherRequestImpl + private static class SimpleScatterGatherRequest implements ScatterGatherRequest { private final BrokerRequest brokerRequest; @@ -270,14 +260,13 @@ private static class ScatterGatherRequestImpl private final long requestTimeoutMs; private final String brokerId; - public ScatterGatherRequestImpl(BrokerRequest request, Map> routingTable, long requestId, long requestTimeoutMs, String brokerId) + public SimpleScatterGatherRequest(BrokerRequest request, Map> routingTable, long requestId, long requestTimeoutMs, String brokerId) { - brokerRequest = request; - this.routingTable = routingTable; - this.requestId = requestId; - - this.requestTimeoutMs = requestTimeoutMs; - this.brokerId = brokerId; + this.brokerRequest = requireNonNull(request); + this.routingTable = requireNonNull(routingTable); + this.requestId = requireNonNull(requestId); + this.requestTimeoutMs = requireNonNull(requestTimeoutMs); + this.brokerId = requireNonNull(brokerId); } @Override @@ -289,13 +278,13 @@ public Map> getRoutingTable() @Override public byte[] getRequestForService(List segments) { - InstanceRequest r = new InstanceRequest(); - r.setRequestId(requestId); - r.setEnableTrace(brokerRequest.isEnableTrace()); - r.setQuery(brokerRequest); - r.setSearchSegments(segments); - r.setBrokerId(brokerId); - return new SerDe(new TCompactProtocol.Factory()).serialize(r); + InstanceRequest request = new InstanceRequest(); + request.setRequestId(requestId); + request.setEnableTrace(brokerRequest.isEnableTrace()); + request.setQuery(brokerRequest); + request.setSearchSegments(segments); + request.setBrokerId(brokerId); + return new SerDe(new TCompactProtocol.Factory()).serialize(request); } @Override diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotSplit.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotSplit.java index 7e2b6069de775..2f1cd1c0fdf47 100644 --- a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotSplit.java +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotSplit.java @@ -17,7 +17,6 @@ import com.facebook.presto.spi.HostAddress; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.airlift.log.Logger; import java.util.List; @@ -26,13 +25,9 @@ public class PinotSplit implements ConnectorSplit { - private static final Logger log = Logger.get(PinotSplit.class); - - private final String connectorId; private final String tableName; private final String host; private final String segment; - private final boolean remotelyAccessible; private final List addresses; private final PinotColumn timeColumn; private final String timeFilter; @@ -41,7 +36,6 @@ public class PinotSplit @JsonCreator public PinotSplit( - @JsonProperty("connectorId") String connectorId, @JsonProperty("tableName") String tableName, @JsonProperty("host") String host, @JsonProperty("segment") String segment, @@ -50,7 +44,6 @@ public PinotSplit( @JsonProperty("pinotFilter") String pinotFilter, @JsonProperty("limit") long limit) { - this.connectorId = requireNonNull(connectorId, "connector id is null"); this.tableName = requireNonNull(tableName, "table name is null"); this.host = requireNonNull(host, "host is null"); this.segment = requireNonNull(segment, "segment is null"); @@ -59,13 +52,6 @@ public PinotSplit( this.pinotFilter = pinotFilter; this.timeFilter = timeFilter; this.limit = limit; - this.remotelyAccessible = true; - } - - @JsonProperty - public String getConnectorId() - { - return connectorId; } @JsonProperty @@ -114,7 +100,7 @@ public long getLimit() public boolean isRemotelyAccessible() { // only http or https is remotely accessible - return remotelyAccessible; + return true; } @Override diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotSplitManager.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotSplitManager.java index f964b298a3f41..93e79254cd720 100644 --- a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotSplitManager.java +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotSplitManager.java @@ -20,50 +20,39 @@ import com.facebook.presto.spi.ConnectorTableLayoutHandle; import com.facebook.presto.spi.FixedSplitSource; import com.facebook.presto.spi.PrestoException; -import com.facebook.presto.spi.block.Block; import com.facebook.presto.spi.connector.ConnectorSplitManager; import com.facebook.presto.spi.connector.ConnectorTransactionHandle; import com.facebook.presto.spi.predicate.Domain; -import com.facebook.presto.spi.predicate.Marker; -import com.facebook.presto.spi.predicate.Range; import com.facebook.presto.spi.predicate.TupleDomain; -import com.facebook.presto.spi.type.VarcharType; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Joiner; import com.google.common.collect.ImmutableList; -import io.airlift.log.Logger; -import io.airlift.slice.Slice; +import org.apache.pinot.common.config.TableNameBuilder; import javax.inject.Inject; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.Stream; import static com.facebook.presto.pinot.PinotErrorCode.PINOT_FAILURE_GETTING_TABLE; -import static com.facebook.presto.pinot.Types.checkType; +import static com.facebook.presto.pinot.PinotQueryBuilder.getColumnPredicate; +import static com.facebook.presto.pinot.PinotQueryBuilder.getTimePredicate; +import static com.facebook.presto.pinot.PinotUtils.TIME_COLUMN_NAME; +import static com.facebook.presto.pinot.PinotUtils.TIME_COLUMN_VALUE; +import static com.facebook.presto.pinot.PinotUtils.checkType; import static com.google.common.base.Preconditions.checkState; import static java.util.Objects.requireNonNull; -import static java.util.stream.Collectors.joining; public class PinotSplitManager implements ConnectorSplitManager { - private static final Logger log = Logger.get(PinotSplitManager.class); - - private final String connectorId; - private final PinotConfig pinotConfig; private final PinotConnection pinotPrestoConnection; @Inject - public PinotSplitManager(PinotConnectorId connectorId, PinotConfig pinotConfig, PinotConnection pinotPrestoConnection) + public PinotSplitManager(PinotConnection pinotPrestoConnection) { - this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); - this.pinotConfig = pinotConfig; this.pinotPrestoConnection = requireNonNull(pinotPrestoConnection, "pinotPrestoConnection is null"); } @@ -76,10 +65,10 @@ public ConnectorSplitSource getSplits( { PinotTableLayoutHandle layoutHandle = checkType(layout, PinotTableLayoutHandle.class, "layout"); PinotTableHandle tableHandle = layoutHandle.getTable(); - PinotTable table = null; - PinotColumn timeColumn = null; - Map>> routingTable = null; - Map timeBoundary = null; + PinotTable table; + PinotColumn timeColumn; + Map>> routingTable; + Map timeBoundary; try { table = pinotPrestoConnection.getTable(tableHandle.getTableName()); timeColumn = pinotPrestoConnection.getPinotTimeColumnForTable(tableHandle.getTableName()); @@ -97,47 +86,15 @@ public ConnectorSplitSource getSplits( List splits = new ArrayList<>(); if (!routingTable.isEmpty()) { - setSplits(splits, timeColumn, routingTable, timeBoundary, getOfflineTableName(tableHandle.getTableName()), tableHandle.getConstraintSummary()); - setSplits(splits, timeColumn, routingTable, timeBoundary, getRealtimeTableName(tableHandle.getTableName()), tableHandle.getConstraintSummary()); + setSplits(splits, timeColumn, routingTable, timeBoundary, TableNameBuilder.OFFLINE.tableNameWithType(tableHandle.getTableName()), tableHandle.getConstraintSummary()); + setSplits(splits, timeColumn, routingTable, timeBoundary, TableNameBuilder.REALTIME.tableNameWithType(tableHandle.getTableName()), tableHandle.getConstraintSummary()); } Collections.shuffle(splits); - log.debug("PinotSplits is %s", Arrays.toString(splits.toArray())); return new FixedSplitSource(splits); } - private String getTimePredicate(String type, String timeColumn, String maxTimeStamp) - { - if ("OFFLINE".equalsIgnoreCase(type)) { - return String.format("%s < %s", timeColumn, maxTimeStamp); - } - if ("REALTIME".equalsIgnoreCase(type)) { - return String.format("%s >= %s", timeColumn, maxTimeStamp); - } - return null; - } - - private void setSplits(List splits, PinotColumn timeColumn, Map>> routingTable, Map timeBoundary, String tableName, TupleDomain constraintSummary) - { - String pinotFilter = getPinotPredicate(constraintSummary); - String timeFilter = ""; - long limit = -1; - if (timeBoundary.containsKey("timeColumnName") && timeBoundary.containsKey("timeColumnValue")) { - timeFilter = getTimePredicate(getTableType(tableName), timeBoundary.get("timeColumnName"), timeBoundary.get("timeColumnValue")); - } - for (String routingTableName : routingTable.keySet()) { - if (routingTableName.equalsIgnoreCase(tableName)) { - Map> hostToSegmentsMap = routingTable.get(routingTableName); - for (String host : hostToSegmentsMap.keySet()) { - for (String segment : hostToSegmentsMap.get(host)) { - splits.add(new PinotSplit(connectorId, routingTableName, host, segment, timeColumn, timeFilter, pinotFilter, limit)); - } - } - } - } - } - /** * Get the predicates for Pinot columns in string format, for constructing Pinot queries directly * Note that for predicates like UDF (WHERE ROUND(fare) > 10), column comparison (WHERE colA - colB > 10, WHERE col/100 > 5), @@ -147,7 +104,7 @@ private void setSplits(List splits, PinotColumn timeColumn, Map< * @return Predicate in Pinot Query Language for Pinot columns */ @VisibleForTesting - String getPinotPredicate(TupleDomain constraintSummary) + static String getPinotPredicate(TupleDomain constraintSummary) { ImmutableList.Builder pinotFilterBuilder = ImmutableList.builder(); @@ -162,126 +119,23 @@ String getPinotPredicate(TupleDomain constraintSummary) return Joiner.on(" AND ").join(pinotFilterBuilder.build()); } - /** - * Get the predicates for a column in string format, for constructing Pinot queries directly - * - * @param domain TupleDomain representing the allowed ranges for a column - * @param columnName Pinot column name - * @return Predicate in Pinot Query Language for the column. Empty string would be returned if no constraints - */ - @VisibleForTesting - String getColumnPredicate(Domain domain, String columnName) - { - List discreteConstraintList = new ArrayList<>(); - List singleValueRangeConstraintList = new ArrayList<>(); - List rangeConstraintList = new ArrayList<>(); - - return domain.getValues().getValuesProcessor().transform( - ranges -> - { - for (Range range : ranges.getOrderedRanges()) { - if (range.isSingleValue()) { - singleValueRangeConstraintList.add(getMarkerValue(range.getLow())); - } - else { - StringBuilder builder = new StringBuilder(); - ImmutableList.Builder bounds = ImmutableList.builder(); - // Get low bound - String equationMark = (range.getLow().getBound() == Marker.Bound.EXACTLY) ? "= " : " "; - if (!range.getLow().isLowerUnbounded()) { - bounds.add(getMarkerValue(range.getLow()) + " <" + equationMark + columnName); - } - // Get high bound - equationMark = (range.getHigh().getBound() == Marker.Bound.EXACTLY) ? "= " : " "; - if (!range.getHigh().isUpperUnbounded()) { - bounds.add(columnName + " <" + equationMark + getMarkerValue(range.getHigh())); - } - // Use AND to combine bounds within the same range - builder.append("(" + Joiner.on(" AND ").join(bounds.build()) + ")"); - rangeConstraintList.add(builder.toString()); - } - } - // Multiple ranges on the same column are OR'ed together. - String rangeConstraint = Joiner.on(" OR ").join(rangeConstraintList); - String discreteConstraint = getDiscretePredicate(true, columnName, singleValueRangeConstraintList); - - return Stream.of(rangeConstraint, discreteConstraint) - .filter(s -> s != null && !s.isEmpty()) - .collect(joining(" OR ")); - }, - discreteValues -> - { - /** - * For some types like {@link com.facebook.presto.type.ColorType} that are not orderable, discreteValues would appear here. - * For most regular types like boolean, char, number, the discrete values would be converted to singleValues in ranges above, - * and would not appear here. So far the column types supported by Pinot all fall in that category. - */ - discreteConstraintList.addAll(discreteValues.getValues().stream().map(Object::toString).collect(Collectors.toList())); - return getDiscretePredicate(discreteValues.isWhiteList(), columnName, discreteConstraintList); - }, - allOrNone -> - { - // no-op - return ""; - }); - } - - /** - * Construct the IN predicate for discrete values - * - * @param isWhitelist true for IN predicate, false for NOT IN predicate - * @param columnName name of the column - * @param discreteConstraintList list of allowed or not allowed values - * @return Stringified clause with IN or NOT IN - */ - String getDiscretePredicate(boolean isWhitelist, String columnName, List discreteConstraintList) - { - if (discreteConstraintList.size() == 0) { - return ""; - } - else { - return columnName + (isWhitelist ? " " : " NOT ") + "IN (" + Joiner.on(',').join(discreteConstraintList) + ")"; - } - } - - /** - * Get the value for the Marker. - * - * @param marker marker in the Domain - * @return Underlying value for the block in the marker. For string, encapsulating quotes will be added. - */ - String getMarkerValue(Marker marker) - { - if (marker.getType() instanceof VarcharType) { - Block highBlock = marker.getValueBlock().get(); - Slice slice = highBlock.getSlice(0, 0, highBlock.getSliceLength(0)); - return "\"" + slice.toStringUtf8() + "\""; - } - else { - return marker.getValue().toString(); - } - } - - private String getOfflineTableName(String table) - { - return table + "_OFFLINE"; - } - - private String getRealtimeTableName(String table) - { - return table + "_REALTIME"; - } - - private String getTableType(String table) + private void setSplits(List splits, PinotColumn timeColumn, Map>> routingTable, Map timeBoundary, String tableName, TupleDomain constraintSummary) { - if (table.endsWith("_REALTIME")) { - return "REALTIME"; + String pinotFilter = getPinotPredicate(constraintSummary); + String timeFilter = ""; + long limit = -1; + if (timeBoundary.containsKey(TIME_COLUMN_NAME) && timeBoundary.containsKey(TIME_COLUMN_VALUE)) { + timeFilter = getTimePredicate(TableNameBuilder.getTableTypeFromTableName(tableName).toString(), timeBoundary.get(TIME_COLUMN_NAME), timeBoundary.get(TIME_COLUMN_VALUE)); } - - if (table.endsWith("_OFFLINE")) { - return "OFFLINE"; + for (String routingTableName : routingTable.keySet()) { + if (routingTableName.equalsIgnoreCase(tableName)) { + Map> hostToSegmentsMap = routingTable.get(routingTableName); + for (String host : hostToSegmentsMap.keySet()) { + for (String segment : hostToSegmentsMap.get(host)) { + splits.add(new PinotSplit(routingTableName, host, segment, timeColumn, timeFilter, pinotFilter, limit)); + } + } + } } - - return null; } } diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotTableHandle.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotTableHandle.java index 06aa798c4a86f..44d55af122921 100644 --- a/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotTableHandle.java +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotTableHandle.java @@ -28,28 +28,19 @@ public final class PinotTableHandle implements ConnectorTableHandle { - private final String connectorId; private final String schemaName; private final String tableName; private TupleDomain constraintSummary; @JsonCreator public PinotTableHandle( - @JsonProperty("connectorId") String connectorId, @JsonProperty("schemaName") String schemaName, @JsonProperty("tableName") String tableName) { - this.connectorId = requireNonNull(connectorId, "connectorId is null"); this.schemaName = requireNonNull(schemaName, "schemaName is null"); this.tableName = requireNonNull(tableName, "tableName is null"); } - @JsonProperty - public String getConnectorId() - { - return connectorId; - } - @JsonProperty public String getSchemaName() { @@ -80,7 +71,7 @@ public SchemaTableName toSchemaTableName() @Override public int hashCode() { - return Objects.hash(connectorId, schemaName, tableName); + return Objects.hash(schemaName, tableName); } @Override @@ -94,12 +85,12 @@ public boolean equals(Object obj) } PinotTableHandle other = (PinotTableHandle) obj; - return Objects.equals(this.connectorId, other.connectorId) && Objects.equals(this.schemaName, other.schemaName) && Objects.equals(this.tableName, other.tableName); + return Objects.equals(this.schemaName, other.schemaName) && Objects.equals(this.tableName, other.tableName); } @Override public String toString() { - return Joiner.on(":").join(connectorId, schemaName, tableName); + return Joiner.on(":").join(schemaName, tableName); } } diff --git a/presto-pinot/src/main/java/com/facebook/presto/pinot/Types.java b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotUtils.java similarity index 65% rename from presto-pinot/src/main/java/com/facebook/presto/pinot/Types.java rename to presto-pinot/src/main/java/com/facebook/presto/pinot/PinotUtils.java index 5bb67c7be98f2..ee3d47e56f73d 100644 --- a/presto-pinot/src/main/java/com/facebook/presto/pinot/Types.java +++ b/presto-pinot/src/main/java/com/facebook/presto/pinot/PinotUtils.java @@ -14,20 +14,21 @@ package com.facebook.presto.pinot; import static com.google.common.base.Preconditions.checkArgument; -import static java.lang.String.format; +import static java.util.Objects.requireNonNull; -final class Types +final class PinotUtils { - private Types() + private PinotUtils() { } + public static final String TIME_COLUMN_NAME = "timeColumnName"; + public static final String TIME_COLUMN_VALUE = "timeColumnValue"; + public static final String QUOTE = "\""; + public static B checkType(A value, Class target, String name) { - if (value == null) { - throw new NullPointerException(format("%s is null", name)); - } - checkArgument(target.isInstance(value), "%s must be of type %s, not %s", name, target.getName(), value.getClass().getName()); + checkArgument(target.isInstance(requireNonNull(value)), "%s must be of type %s, not %s", name, target.getName(), value.getClass().getName()); return target.cast(value); } } diff --git a/presto-pinot/src/test/java/com/facebook/presto/pinot/PinotQueryGeneratorTest.java b/presto-pinot/src/test/java/com/facebook/presto/pinot/PinotQueryGeneratorTest.java deleted file mode 100644 index 0517a80b7a353..0000000000000 --- a/presto-pinot/src/test/java/com/facebook/presto/pinot/PinotQueryGeneratorTest.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.facebook.presto.pinot; - -import com.facebook.presto.testing.assertions.Assert; -import org.testng.annotations.BeforeTest; -import org.testng.annotations.Test; - -import java.util.ArrayList; -import java.util.List; - -import static com.facebook.presto.pinot.PinotQueryGenerator.getPinotQuery; -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; - -public class PinotQueryGeneratorTest -{ - final List columnHandles = new ArrayList<>(); - - @BeforeTest - void init() - { - columnHandles.add(new PinotColumnHandle("pinot", "varchar", VARCHAR, 0)); - columnHandles.add(new PinotColumnHandle("pinot", "int", INTEGER, 1)); - columnHandles.add(new PinotColumnHandle("pinot", "secondsSinceEpoch", BIGINT, 2)); - columnHandles.add(new PinotColumnHandle("pinot", "boolean", BOOLEAN, 3)); - columnHandles.add(new PinotColumnHandle("pinot", "double", DOUBLE, 4)); - } - - @Test - public void testGetPinotQuerySelectAll() - { - String expectedQuery = "SELECT varchar, int, secondsSinceEpoch, boolean, double FROM table LIMIT 10"; - Assert.assertEquals(expectedQuery, getPinotQuery(new PinotConfig(), columnHandles, "", "", "table", 10)); - } - - @Test - public void testGetPinotQueryWithPredicate() - { - String expectedQuery = "SELECT varchar, int, secondsSinceEpoch, boolean, double FROM table WHERE ((int > 3)) AND ((secondsSinceEpoch > 10000)) LIMIT 10"; - Assert.assertEquals(expectedQuery, getPinotQuery(new PinotConfig(), columnHandles, "(int > 3)", "(secondsSinceEpoch > 10000)", "table", 10)); - } -} diff --git a/presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotColumnHandle.java b/presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotColumnHandle.java index 087c1bb086a15..914cf78d04b03 100644 --- a/presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotColumnHandle.java +++ b/presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotColumnHandle.java @@ -16,14 +16,14 @@ import io.airlift.testing.EquivalenceTester; import org.testng.annotations.Test; -import static com.facebook.presto.pinot.MetadataUtil.COLUMN_CODEC; +import static com.facebook.presto.pinot.TestPinotMetadataUtil.COLUMN_CODEC; 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("connectorId", "columnName", VARCHAR, 0); + private static final PinotColumnHandle columnHandle = new PinotColumnHandle("columnName", VARCHAR, 0); @Test public void testJsonRoundTrip() @@ -39,20 +39,15 @@ public void testEquivalence() EquivalenceTester .equivalenceTester() .addEquivalentGroup( - new PinotColumnHandle("connectorId", "columnName", VARCHAR, 0), - new PinotColumnHandle("connectorId", "columnName", VARCHAR, 0), - new PinotColumnHandle("connectorId", "columnName", BIGINT, 0), - new PinotColumnHandle("connectorId", "columnName", VARCHAR, 1)) + new PinotColumnHandle("columnName", VARCHAR, 0), + new PinotColumnHandle("columnName", VARCHAR, 0), + new PinotColumnHandle("columnName", BIGINT, 0), + new PinotColumnHandle("columnName", VARCHAR, 1)) .addEquivalentGroup( - new PinotColumnHandle("connectorIdX", "columnName", VARCHAR, 0), - new PinotColumnHandle("connectorIdX", "columnName", VARCHAR, 0), - new PinotColumnHandle("connectorIdX", "columnName", BIGINT, 0), - new PinotColumnHandle("connectorIdX", "columnName", VARCHAR, 1)) - .addEquivalentGroup( - new PinotColumnHandle("connectorId", "columnNameX", VARCHAR, 0), - new PinotColumnHandle("connectorId", "columnNameX", VARCHAR, 0), - new PinotColumnHandle("connectorId", "columnNameX", BIGINT, 0), - new PinotColumnHandle("connectorId", "columnNameX", VARCHAR, 1)) + new PinotColumnHandle("columnNameX", VARCHAR, 0), + new PinotColumnHandle("columnNameX", VARCHAR, 0), + new PinotColumnHandle("columnNameX", BIGINT, 0), + new PinotColumnHandle("columnNameX", VARCHAR, 1)) .check(); } } diff --git a/presto-pinot/src/test/java/com/facebook/presto/pinot/MetadataUtil.java b/presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotMetadataUtil.java similarity index 88% rename from presto-pinot/src/test/java/com/facebook/presto/pinot/MetadataUtil.java rename to presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotMetadataUtil.java index 21e52898eaeb6..980a7d9ccc264 100644 --- a/presto-pinot/src/test/java/com/facebook/presto/pinot/MetadataUtil.java +++ b/presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotMetadataUtil.java @@ -33,14 +33,25 @@ import static com.facebook.presto.spi.type.VarcharType.VARCHAR; import static io.airlift.json.JsonCodec.listJsonCodec; import static java.util.Locale.ENGLISH; +import static java.util.Objects.requireNonNull; -public final class MetadataUtil +public final class TestPinotMetadataUtil { - public static final JsonCodec>> CATALOG_CODEC; - public static final JsonCodec TABLE_CODEC; public static final JsonCodec COLUMN_CODEC; - private MetadataUtil() + private static final JsonCodec>> CATALOG_CODEC; + private static final JsonCodec TABLE_CODEC; + + 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); + } + + private TestPinotMetadataUtil() { } @@ -58,19 +69,7 @@ public TestingTypeDeserializer() protected Type _deserialize(String value, DeserializationContext context) { Type type = types.get(value.toLowerCase(ENGLISH)); - if (type == null) { - throw new IllegalArgumentException(String.valueOf("Unknown type " + value)); - } - return type; + return requireNonNull(type); } } - - 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); - } } diff --git a/presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotQueryBuilder.java b/presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotQueryBuilder.java new file mode 100644 index 0000000000000..6ef434a6d7b12 --- /dev/null +++ b/presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotQueryBuilder.java @@ -0,0 +1,116 @@ +/* + * 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.predicate.Domain; +import com.facebook.presto.spi.predicate.Range; +import com.facebook.presto.spi.predicate.SortedRangeSet; +import com.facebook.presto.spi.predicate.ValueSet; +import com.facebook.presto.testing.assertions.Assert; +import org.testng.annotations.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static com.facebook.presto.pinot.PinotQueryBuilder.getColumnPredicate; +import static com.facebook.presto.pinot.PinotQueryBuilder.getDiscretePredicate; +import static com.facebook.presto.pinot.PinotQueryBuilder.getPinotQuery; +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; + +public class TestPinotQueryBuilder +{ + private static List columnHandles = Arrays.asList( + new PinotColumnHandle("varchar", VARCHAR, 0), + new PinotColumnHandle("int", INTEGER, 1), + new PinotColumnHandle("secondsSinceEpoch", BIGINT, 2), + new PinotColumnHandle("boolean", BOOLEAN, 3), + new PinotColumnHandle("double", DOUBLE, 4)); + + @Test + public void testGetPinotQuerySelectAll() + { + String expectedQuery = "SELECT varchar, int, secondsSinceEpoch, boolean, double FROM table LIMIT 10"; + Assert.assertEquals(expectedQuery, getPinotQuery(new PinotConfig(), columnHandles, "", "", "table", 10)); + } + + @Test + public void testGetPinotQueryWithPredicate() + { + String expectedQuery = "SELECT varchar, int, secondsSinceEpoch, boolean, double FROM table WHERE ((int > 3)) AND ((secondsSinceEpoch > 10000)) LIMIT 10"; + Assert.assertEquals(expectedQuery, getPinotQuery(new PinotConfig(), columnHandles, "(int > 3)", "(secondsSinceEpoch > 10000)", "table", 10)); + } + + @Test + public void testSingleValueRanges() + { + Domain domain = com.facebook.presto.spi.predicate.Domain.multipleValues(BIGINT, new ArrayList<>(Arrays.asList(1L, 10L))); + String expectedFilter = "int IN (1,10)"; + + Assert.assertEquals(expectedFilter, getColumnPredicate(domain, columnHandles.get(1).getColumnName())); + } + + @Test + public void testRangeValues() + { + Domain domain = Domain.create(ValueSet.ofRanges( + Range.greaterThan(BIGINT, 1L).intersect(Range.lessThan(BIGINT, 10L))), false); + + String expectedFilter = "(1 < int AND int < 10)"; + Assert.assertEquals(expectedFilter, getColumnPredicate(domain, columnHandles.get(1).getColumnName())); + } + + @Test + public void testOneSideRanges() + { + Domain domain = Domain.create(ValueSet.ofRanges( + Range.lessThanOrEqual(BIGINT, 10L)), false); + + String expectedFilter = "(int <= 10)"; + Assert.assertEquals(expectedFilter, getColumnPredicate(domain, columnHandles.get(1).getColumnName())); + } + + @Test + public void testMultipleRanges() + { + Domain domain = Domain.create(ValueSet.ofRanges( + Range.equal(BIGINT, 20L), + Range.greaterThan(BIGINT, 1L).intersect(Range.lessThan(BIGINT, 10L)), + Range.greaterThan(BIGINT, 12L).intersect(Range.lessThan(BIGINT, 18L))), false); + + String expectedFilter = "(1 < int AND int < 10) OR (12 < int AND int < 18) OR int IN (20)"; + Assert.assertEquals(expectedFilter, getColumnPredicate(domain, columnHandles.get(1).getColumnName())); + } + + @Test + public void testGetDiscretePredicate() + { + Assert.assertEquals("", getDiscretePredicate(true, "int", new ArrayList<>())); + Assert.assertEquals("int IN (1,2)", getDiscretePredicate(true, "int", new ArrayList<>(Arrays.asList("1", "2")))); + Assert.assertEquals("int NOT IN (1,2)", getDiscretePredicate(false, "int", new ArrayList<>(Arrays.asList("1", "2")))); + Assert.assertEquals("varchar NOT IN (\"cn\",\"us\")", getDiscretePredicate(false, "varchar", new ArrayList<>(Arrays.asList("\"cn\"", "\"us\"")))); + } + + @Test + public void testEmptyDomain() + { + SortedRangeSet sortedRangeSet = SortedRangeSet.copyOf(BIGINT, new ArrayList<>()); + Domain domain = Domain.create(sortedRangeSet, false); + Assert.assertEquals("", getColumnPredicate(domain, columnHandles.get(0).getColumnName())); + } +} diff --git a/presto-pinot/src/test/java/com/facebook/presto/pinot/PinotSplitManagerTest.java b/presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotSplitManager.java similarity index 54% rename from presto-pinot/src/test/java/com/facebook/presto/pinot/PinotSplitManagerTest.java rename to presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotSplitManager.java index eeb91517ad677..6827ea53dca8f 100644 --- a/presto-pinot/src/test/java/com/facebook/presto/pinot/PinotSplitManagerTest.java +++ b/presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotSplitManager.java @@ -17,18 +17,12 @@ import com.facebook.presto.spi.predicate.Domain; import com.facebook.presto.spi.predicate.EquatableValueSet; import com.facebook.presto.spi.predicate.Range; -import com.facebook.presto.spi.predicate.SortedRangeSet; import com.facebook.presto.spi.predicate.TupleDomain; import com.facebook.presto.spi.predicate.ValueSet; import com.facebook.presto.testing.assertions.Assert; import io.airlift.slice.Slices; -import org.testng.annotations.BeforeTest; import org.testng.annotations.Test; -import java.net.SocketException; -import java.net.UnknownHostException; -import java.util.ArrayList; -import java.util.Arrays; import java.util.HashMap; import java.util.HashSet; import java.util.Map; @@ -37,62 +31,12 @@ import static com.facebook.presto.spi.type.VarcharType.VARCHAR; import static com.facebook.presto.type.ColorType.COLOR; -public class PinotSplitManagerTest +public class TestPinotSplitManager { - PinotSplitManager pinotSplitManager; - PinotColumnHandle columnCityId; - PinotColumnHandle columnCountryName; - PinotColumnHandle columnColor; - - @BeforeTest - void init() throws SocketException, UnknownHostException - { - pinotSplitManager = new PinotSplitManager(new PinotConnectorId(""), new PinotConfig(), new PinotConnection(new PinotClusterInfoFetcher(new PinotConfig()))); - columnCityId = new PinotColumnHandle("pinot", "city_id", BIGINT, 0); - columnCountryName = new PinotColumnHandle("pinot", "country_name", VARCHAR, 1); - columnColor = new PinotColumnHandle("pinot", "color", COLOR, 2); - } - - @Test - public void testSingleValueRanges() - { - Domain domain = com.facebook.presto.spi.predicate.Domain.multipleValues(BIGINT, new ArrayList<>(Arrays.asList(1L, 10L))); - String expectedFilter = "city_id IN (1,10)"; - - Assert.assertEquals(expectedFilter, pinotSplitManager.getColumnPredicate(domain, columnCityId.getColumnName())); - } - - @Test - public void testRangeValues() - { - Domain domain = Domain.create(ValueSet.ofRanges( - Range.greaterThan(BIGINT, 1L).intersect(Range.lessThan(BIGINT, 10L))), false); - - String expectedFilter = "(1 < city_id AND city_id < 10)"; - Assert.assertEquals(expectedFilter, pinotSplitManager.getColumnPredicate(domain, columnCityId.getColumnName())); - } - - @Test - public void testOneSideRanges() - { - Domain domain = Domain.create(ValueSet.ofRanges( - Range.lessThanOrEqual(BIGINT, 10L)), false); - - String expectedFilter = "(city_id <= 10)"; - Assert.assertEquals(expectedFilter, pinotSplitManager.getColumnPredicate(domain, columnCityId.getColumnName())); - } - - @Test - public void testMultipleRanges() - { - Domain domain = Domain.create(ValueSet.ofRanges( - Range.equal(BIGINT, 20L), - Range.greaterThan(BIGINT, 1L).intersect(Range.lessThan(BIGINT, 10L)), - Range.greaterThan(BIGINT, 12L).intersect(Range.lessThan(BIGINT, 18L))), false); - - String expectedFilter = "(1 < city_id AND city_id < 10) OR (12 < city_id AND city_id < 18) OR city_id IN (20)"; - Assert.assertEquals(expectedFilter, pinotSplitManager.getColumnPredicate(domain, columnCityId.getColumnName())); - } + private static final PinotSplitManager pinotSplitManager = new PinotSplitManager(new PinotConnection(new PinotClusterInfoFetcher(new PinotConfig()))); + private static final PinotColumnHandle columnCityId = new PinotColumnHandle("city_id", BIGINT, 0); + private static final PinotColumnHandle columnCountryName = new PinotColumnHandle("country_name", VARCHAR, 1); + private static final PinotColumnHandle columnColor = new PinotColumnHandle("color", COLOR, 2); @Test public void testMultipleColumns() @@ -152,27 +96,9 @@ public void testNotPredicateInRanges() Assert.assertEquals(expectedFilter, pinotSplitManager.getPinotPredicate(constraintSummary)); } - @Test - public void testEmptyDomain() - { - SortedRangeSet sortedRangeSet = SortedRangeSet.copyOf(BIGINT, new ArrayList<>()); - Domain domain = Domain.create(sortedRangeSet, false); - - Assert.assertEquals("", pinotSplitManager.getColumnPredicate(domain, columnCityId.getColumnName())); - } - @Test public void testEmptyConstraintSummary() { Assert.assertEquals("", pinotSplitManager.getPinotPredicate(TupleDomain.all())); } - - @Test - public void testGetDiscretePredicate() - { - Assert.assertEquals("", pinotSplitManager.getDiscretePredicate(true, "city_id", new ArrayList<>())); - Assert.assertEquals("city_id IN (1,2)", pinotSplitManager.getDiscretePredicate(true, "city_id", new ArrayList<>(Arrays.asList("1", "2")))); - Assert.assertEquals("city_id NOT IN (1,2)", pinotSplitManager.getDiscretePredicate(false, "city_id", new ArrayList<>(Arrays.asList("1", "2")))); - Assert.assertEquals("country_name NOT IN (\"cn\",\"us\")", pinotSplitManager.getDiscretePredicate(false, "country_name", new ArrayList<>(Arrays.asList("\"cn\"", "\"us\"")))); - } } diff --git a/presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotTableHandle.java b/presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotTableHandle.java index f29c9f66abb8f..400836a163e4f 100644 --- a/presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotTableHandle.java +++ b/presto-pinot/src/test/java/com/facebook/presto/pinot/TestPinotTableHandle.java @@ -22,7 +22,7 @@ public class TestPinotTableHandle { - private final PinotTableHandle tableHandle = new PinotTableHandle("connectorId", "schemaName", "tableName"); + private final PinotTableHandle tableHandle = new PinotTableHandle("schemaName", "tableName"); @Test public void testJsonRoundTrip() @@ -36,6 +36,10 @@ public void testJsonRoundTrip() @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(); + EquivalenceTester.equivalenceTester() + .addEquivalentGroup(new PinotTableHandle("schema", "table"), new PinotTableHandle("schema", "table")) + .addEquivalentGroup(new PinotTableHandle("schemaX", "table"), new PinotTableHandle("schemaX", "table")) + .addEquivalentGroup(new PinotTableHandle("schema", "tableX"), new PinotTableHandle("schema", "tableX")) + .check(); } }