diff --git a/.gitignore b/.gitignore
index f1d9472f81de7..7d4f797920d9f 100644
--- a/.gitignore
+++ b/.gitignore
@@ -23,3 +23,4 @@ benchmark_outputs
*.class
.checkstyle
.mvn/timing.properties
+.editorconfig
diff --git a/.travis.yml b/.travis.yml
index 9b48214fd8ece..96bacd1d25bae 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -16,11 +16,17 @@ env:
- TEST_SPECIFIC_MODULES=presto-cassandra
- TEST_SPECIFIC_MODULES=presto-hive
- TEST_OTHER_MODULES=!presto-tests,!presto-raptor,!presto-accumulo,!presto-cassandra,!presto-hive,!presto-docs,!presto-server,!presto-server-rpm
- - PRODUCT_TESTS=true
+ - PRODUCT_TESTS_BASIC_ENVIRONMENT=true
+ - PRODUCT_TESTS_SPECIFIC_ENVIRONMENT=true
- HIVE_TESTS=true
sudo: required
dist: trusty
+group: deprecated-2017Q2
+addons:
+ apt:
+ packages:
+ - oracle-java8-installer
cache:
directories:
@@ -40,7 +46,7 @@ install:
./mvnw install $MAVEN_FAST_INSTALL -pl '!presto-docs,!presto-server,!presto-server-rpm'
fi
- |
- if [[ -v PRODUCT_TESTS ]]; then
+ if [[ -v PRODUCT_TESTS_BASIC_ENVIRONMENT || -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT ]]; then
./mvnw install $MAVEN_FAST_INSTALL -pl '!presto-docs,!presto-server-rpm'
fi
- |
@@ -62,28 +68,33 @@ script:
./mvnw test $MAVEN_SKIP_CHECKS_AND_DOCS -B -pl $TEST_OTHER_MODULES
fi
- |
- if [[ -v PRODUCT_TESTS ]]; then
+ if [[ -v PRODUCT_TESTS_BASIC_ENVIRONMENT ]]; then
presto-product-tests/bin/run_on_docker.sh \
multinode -x quarantine,big_query,storage_formats,profile_specific_tests,tpcds
fi
- |
- if [[ -v PRODUCT_TESTS ]]; then
+ if [[ -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT ]]; then
presto-product-tests/bin/run_on_docker.sh \
singlenode-kerberos-hdfs-impersonation -g storage_formats,cli,hdfs_impersonation,authorization
fi
- |
- if [[ -v PRODUCT_TESTS ]]; then
+ if [[ -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT ]]; then
presto-product-tests/bin/run_on_docker.sh \
- singlenode-ldap -g ldap_cli
+ singlenode-ldap -g ldap -x simba_jdbc
fi
# SQL server image sporadically hangs during the startup
# TODO: Uncomment it once issue is fixed
# https://github.com/Microsoft/mssql-docker/issues/76
# - |
-# if [[ -v PRODUCT_TESTS ]]; then
+# if [[ -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT ]]; then
# presto-product-tests/bin/run_on_docker.sh \
# singlenode-sqlserver -g sqlserver
# fi
+ - |
+ if [[ -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT ]]; then
+ presto-product-tests/bin/run_on_docker.sh \
+ multinode-tls -g smoke,cli,group-by,join,tls
+ fi
- |
if [[ -v HIVE_TESTS ]]; then
presto-hive-hadoop2/bin/run_on_docker.sh
diff --git a/README.md b/README.md
index 7df5380c68c48..acfe40570b3da 100644
--- a/README.md
+++ b/README.md
@@ -15,13 +15,13 @@ See the [User Manual](https://prestodb.io/docs/current/) for deployment instruct
Presto is a standard Maven project. Simply run the following command from the project root directory:
- mvn clean install
+ ./mvnw clean install
On the first build, Maven will download all the dependencies from the internet and cache them in the local repository (`~/.m2/repository`), which can take a considerable amount of time. Subsequent builds will be faster.
Presto has a comprehensive set of unit tests that can take several minutes to run. You can disable the tests when building:
- mvn clean install -DskipTests
+ ./mvnw clean install -DskipTests
## Running Presto in your IDE
diff --git a/pom.xml b/pom.xml
index c7eb600de48c7..e733d6a8c3d60 100644
--- a/pom.xml
+++ b/pom.xml
@@ -5,12 +5,12 @@
io.airlift
airbase
- 62
+ 64
com.facebook.presto
presto-root
- 0.175-SNAPSHOT
+ 0.180-SNAPSHOT
pom
presto-root
@@ -47,15 +47,15 @@
3.3.9
4.6
- 0.145
+ 0.148
${dep.airlift.version}
0.29
1.11.30
- 1.30
+ 3.8.1
+ 1.31
6.10
-
- true
- None
+ 0.15.1
+ 0.15.2
Asia/Katmandu
@@ -81,10 +81,7 @@
presto-orc
presto-rcfile
presto-hive
- presto-hive-hadoop1
presto-hive-hadoop2
- presto-hive-cdh4
- presto-hive-cdh5
presto-teradata-functions
presto-example-http
presto-local-file
@@ -114,6 +111,9 @@
presto-plugin-toolkit
presto-resource-group-managers
presto-benchto-benchmarks
+ presto-thrift-connector-api
+ presto-thrift-testing-server
+ presto-thrift-connector
@@ -180,13 +180,6 @@
${project.version}
-
- com.facebook.presto
- presto-hive-cdh4
- ${project.version}
- zip
-
-
com.facebook.presto
presto-example-http
@@ -337,32 +330,46 @@
com.facebook.presto.hadoop
- hadoop-apache1
- 0.4
+ hadoop-apache2
+ 2.7.3-1
- com.facebook.presto.hadoop
- hadoop-apache2
- 0.10
+ com.facebook.presto.hive
+ hive-apache
+ 1.2.0-2
- com.facebook.presto.hadoop
- hadoop-cdh4
- 0.10
+ com.facebook.presto.orc
+ orc-protobuf
+ 3
- com.facebook.presto.hive
- hive-apache
- 1.2.0-1
+ com.facebook.presto
+ presto-thrift-connector-api
+ ${project.version}
- com.facebook.presto.orc
- orc-protobuf
- 2
+ com.facebook.presto
+ presto-thrift-connector-api
+ ${project.version}
+ test-jar
+
+
+
+ com.facebook.presto
+ presto-thrift-testing-server
+ ${project.version}
+
+
+
+ com.facebook.presto
+ presto-thrift-connector
+ ${project.version}
+ zip
@@ -374,7 +381,7 @@
io.airlift
aircompressor
- 0.5
+ 0.7
@@ -455,6 +462,12 @@
${dep.airlift.version}
+
+ io.airlift
+ jaxrs-testing
+ ${dep.airlift.version}
+
+
io.airlift
jmx
@@ -566,7 +579,7 @@
mysql
mysql-connector-java
- 5.1.35
+ 5.1.41
@@ -619,6 +632,54 @@
2.78
+
+ com.squareup.okhttp3
+ okhttp
+ ${dep.okhttp.version}
+
+
+
+ com.squareup.okhttp3
+ mockwebserver
+ ${dep.okhttp.version}
+
+
+
+ com.facebook.swift
+ swift-annotations
+ ${dep.swift.version}
+
+
+
+ com.facebook.swift
+ swift-codec
+ ${dep.swift.version}
+
+
+
+ com.facebook.swift
+ swift-service
+ ${dep.swift.version}
+
+
+
+ com.facebook.swift
+ swift-javadoc
+ ${dep.swift.version}
+
+
+
+ com.facebook.nifty
+ nifty-core
+ ${dep.nifty.version}
+
+
+
+ com.facebook.nifty
+ nifty-client
+ ${dep.nifty.version}
+
+
org.apache.thrift
libthrift
@@ -897,7 +958,7 @@
org.codehaus.mojo
exec-maven-plugin
- 1.2.1
+ 1.6.0
@@ -919,6 +980,27 @@
+
+ com.ning.maven.plugins
+ maven-dependency-versions-check-plugin
+
+
+
+ com.google.inject
+ guice
+ 4.0-beta5
+ 4.0
+
+
+ com.google.inject.extensions
+ guice-multibindings
+ 4.0-beta5
+ 4.0
+
+
+
+
+
@@ -1094,26 +1176,6 @@
-
-
- cli
-
-
-
- org.codehaus.mojo
- exec-maven-plugin
-
- ${cli.skip-execute}
- ${java.home}/bin/java
- ${cli.main-class}
-
- --debug
-
-
-
-
-
-
eclipse-compiler
diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml
index e3854c50ab730..4645eb6b3bdb9 100644
--- a/presto-accumulo/pom.xml
+++ b/presto-accumulo/pom.xml
@@ -5,7 +5,7 @@
com.facebook.presto
presto-root
- 0.175-SNAPSHOT
+ 0.180-SNAPSHOT
presto-accumulo
@@ -317,5 +317,11 @@
testng
test
+
+
+ javax.annotation
+ javax.annotation-api
+ test
+
diff --git a/presto-accumulo/src/main/java/com/facebook/presto/accumulo/AccumuloClient.java b/presto-accumulo/src/main/java/com/facebook/presto/accumulo/AccumuloClient.java
index da91ef8c396b6..d3c9cff3683fa 100644
--- a/presto-accumulo/src/main/java/com/facebook/presto/accumulo/AccumuloClient.java
+++ b/presto-accumulo/src/main/java/com/facebook/presto/accumulo/AccumuloClient.java
@@ -101,17 +101,17 @@ public AccumuloClient(
Connector connector,
AccumuloConfig config,
ZooKeeperMetadataManager metaManager,
- AccumuloTableManager tableManager)
+ AccumuloTableManager tableManager,
+ IndexLookup indexLookup)
throws AccumuloException, AccumuloSecurityException
{
this.connector = requireNonNull(connector, "connector is null");
this.username = requireNonNull(config, "config is null").getUsername();
this.metaManager = requireNonNull(metaManager, "metaManager is null");
this.tableManager = requireNonNull(tableManager, "tableManager is null");
- this.auths = connector.securityOperations().getUserAuthorizations(username);
+ this.indexLookup = requireNonNull(indexLookup, "indexLookup is null");
- // Create the index lookup utility
- this.indexLookup = new IndexLookup(connector, config, this.auths);
+ this.auths = connector.securityOperations().getUserAuthorizations(username);
}
public AccumuloTable createTable(ConnectorTableMetadata meta)
@@ -440,9 +440,6 @@ public void dropTable(AccumuloTable table)
{
SchemaTableName tableName = new SchemaTableName(table.getSchema(), table.getTable());
- // Drop cardinality cache from index lookup
- indexLookup.dropCache(tableName.getSchemaName(), tableName.getTableName());
-
// Remove the table metadata from Presto
if (metaManager.getTable(tableName) != null) {
metaManager.deleteTableMetadata(tableName);
diff --git a/presto-accumulo/src/main/java/com/facebook/presto/accumulo/AccumuloModule.java b/presto-accumulo/src/main/java/com/facebook/presto/accumulo/AccumuloModule.java
index 60993e8833a46..32529b4023532 100644
--- a/presto-accumulo/src/main/java/com/facebook/presto/accumulo/AccumuloModule.java
+++ b/presto-accumulo/src/main/java/com/facebook/presto/accumulo/AccumuloModule.java
@@ -16,6 +16,8 @@
import com.facebook.presto.accumulo.conf.AccumuloConfig;
import com.facebook.presto.accumulo.conf.AccumuloSessionProperties;
import com.facebook.presto.accumulo.conf.AccumuloTableProperties;
+import com.facebook.presto.accumulo.index.ColumnCardinalityCache;
+import com.facebook.presto.accumulo.index.IndexLookup;
import com.facebook.presto.accumulo.io.AccumuloPageSinkProvider;
import com.facebook.presto.accumulo.io.AccumuloRecordSetProvider;
import com.facebook.presto.accumulo.metadata.AccumuloTable;
@@ -96,6 +98,8 @@ public void configure(Binder binder)
binder.bind(AccumuloTableProperties.class).in(Scopes.SINGLETON);
binder.bind(ZooKeeperMetadataManager.class).in(Scopes.SINGLETON);
binder.bind(AccumuloTableManager.class).in(Scopes.SINGLETON);
+ binder.bind(IndexLookup.class).in(Scopes.SINGLETON);
+ binder.bind(ColumnCardinalityCache.class).in(Scopes.SINGLETON);
binder.bind(Connector.class).toProvider(ConnectorProvider.class);
configBinder(binder).bindConfig(AccumuloConfig.class);
diff --git a/presto-accumulo/src/main/java/com/facebook/presto/accumulo/conf/AccumuloConfig.java b/presto-accumulo/src/main/java/com/facebook/presto/accumulo/conf/AccumuloConfig.java
index 4330f3619cbb4..d3e09b15a27dc 100644
--- a/presto-accumulo/src/main/java/com/facebook/presto/accumulo/conf/AccumuloConfig.java
+++ b/presto-accumulo/src/main/java/com/facebook/presto/accumulo/conf/AccumuloConfig.java
@@ -15,6 +15,7 @@
import io.airlift.configuration.Config;
import io.airlift.configuration.ConfigDescription;
+import io.airlift.configuration.ConfigSecuritySensitive;
import io.airlift.units.Duration;
import javax.validation.constraints.Min;
@@ -92,6 +93,7 @@ public String getPassword()
}
@Config(PASSWORD)
+ @ConfigSecuritySensitive
@ConfigDescription("Sets the password for the configured user")
public AccumuloConfig setPassword(String password)
{
diff --git a/presto-accumulo/src/main/java/com/facebook/presto/accumulo/conf/AccumuloSessionProperties.java b/presto-accumulo/src/main/java/com/facebook/presto/accumulo/conf/AccumuloSessionProperties.java
index 255204a21e61e..d266cbebb2ffc 100644
--- a/presto-accumulo/src/main/java/com/facebook/presto/accumulo/conf/AccumuloSessionProperties.java
+++ b/presto-accumulo/src/main/java/com/facebook/presto/accumulo/conf/AccumuloSessionProperties.java
@@ -16,6 +16,7 @@
import com.facebook.presto.spi.ConnectorSession;
import com.facebook.presto.spi.session.PropertyMetadata;
import com.google.common.collect.ImmutableList;
+import io.airlift.units.Duration;
import javax.inject.Inject;
@@ -25,6 +26,7 @@
import static com.facebook.presto.spi.session.PropertyMetadata.doubleSessionProperty;
import static com.facebook.presto.spi.session.PropertyMetadata.integerSessionProperty;
import static com.facebook.presto.spi.session.PropertyMetadata.stringSessionProperty;
+import static com.facebook.presto.spi.type.VarcharType.VARCHAR;
/**
* Class contains all session-based properties for the Accumulo connector.
@@ -44,6 +46,8 @@ public final class AccumuloSessionProperties
private static final String INDEX_LOWEST_CARDINALITY_THRESHOLD = "index_lowest_cardinality_threshold";
private static final String INDEX_METRICS_ENABLED = "index_metrics_enabled";
private static final String SCAN_USERNAME = "scan_username";
+ private static final String INDEX_SHORT_CIRCUIT_CARDINALITY_FETCH = "index_short_circuit_cardinality_fetch";
+ private static final String INDEX_CARDINALITY_CACHE_POLLING_DURATION = "index_cardinality_cache_polling_duration";
private final List> sessionProperties;
@@ -94,7 +98,22 @@ public AccumuloSessionProperties()
true,
false);
- sessionProperties = ImmutableList.of(s1, s2, s3, s4, s5, s6, s7, s8);
+ PropertyMetadata s9 = booleanSessionProperty(
+ INDEX_SHORT_CIRCUIT_CARDINALITY_FETCH,
+ "Short circuit the retrieval of index metrics once any column is less than the lowest cardinality threshold. Default true",
+ true,
+ false);
+
+ PropertyMetadata s10 = new PropertyMetadata<>(
+ INDEX_CARDINALITY_CACHE_POLLING_DURATION,
+ "Sets the cardinality cache polling duration for short circuit retrieval of index metrics. Default 10ms",
+ VARCHAR, String.class,
+ "10ms",
+ false,
+ duration -> Duration.valueOf(duration.toString()).toString(),
+ object -> object);
+
+ sessionProperties = ImmutableList.of(s1, s2, s3, s4, s5, s6, s7, s8, s9, s10);
}
public List> getSessionProperties()
@@ -132,6 +151,11 @@ public static double getIndexSmallCardThreshold(ConnectorSession session)
return session.getProperty(INDEX_LOWEST_CARDINALITY_THRESHOLD, Double.class);
}
+ public static Duration getIndexCardinalityCachePollingDuration(ConnectorSession session)
+ {
+ return Duration.valueOf(session.getProperty(INDEX_CARDINALITY_CACHE_POLLING_DURATION, String.class));
+ }
+
public static boolean isIndexMetricsEnabled(ConnectorSession session)
{
return session.getProperty(INDEX_METRICS_ENABLED, Boolean.class);
@@ -141,4 +165,9 @@ public static String getScanUsername(ConnectorSession session)
{
return session.getProperty(SCAN_USERNAME, String.class);
}
+
+ public static boolean isIndexShortCircuitEnabled(ConnectorSession session)
+ {
+ return session.getProperty(INDEX_SHORT_CIRCUIT_CARDINALITY_FETCH, Boolean.class);
+ }
}
diff --git a/presto-accumulo/src/main/java/com/facebook/presto/accumulo/index/ColumnCardinalityCache.java b/presto-accumulo/src/main/java/com/facebook/presto/accumulo/index/ColumnCardinalityCache.java
index f4afef21a5dfa..8474240c7ffc3 100644
--- a/presto-accumulo/src/main/java/com/facebook/presto/accumulo/index/ColumnCardinalityCache.java
+++ b/presto-accumulo/src/main/java/com/facebook/presto/accumulo/index/ColumnCardinalityCache.java
@@ -14,39 +14,63 @@
package com.facebook.presto.accumulo.index;
import com.facebook.presto.accumulo.conf.AccumuloConfig;
-import com.facebook.presto.accumulo.metadata.AccumuloTable;
import com.facebook.presto.accumulo.model.AccumuloColumnConstraint;
+import com.facebook.presto.spi.PrestoException;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
-import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableMultimap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.ListMultimap;
import com.google.common.collect.Multimap;
-import com.google.common.collect.TreeMultimap;
+import com.google.common.collect.MultimapBuilder;
+import io.airlift.concurrent.BoundedExecutor;
import io.airlift.log.Logger;
import io.airlift.units.Duration;
import org.apache.accumulo.core.client.BatchScanner;
import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Scanner;
import org.apache.accumulo.core.client.TableNotFoundException;
import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.PartialKey;
import org.apache.accumulo.core.data.Range;
import org.apache.accumulo.core.data.Value;
import org.apache.accumulo.core.security.Authorizations;
+import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.io.Text;
-import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.Nonnull;
+import javax.annotation.PreDestroy;
+import javax.inject.Inject;
import java.util.Collection;
import java.util.HashMap;
import java.util.Map;
import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.CompletionService;
import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeUnit;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.function.Function;
import java.util.stream.Collectors;
+import static com.facebook.presto.accumulo.AccumuloErrorCode.UNEXPECTED_ACCUMULO_ERROR;
+import static com.facebook.presto.accumulo.index.Indexer.CARDINALITY_CQ_AS_TEXT;
+import static com.facebook.presto.accumulo.index.Indexer.getIndexColumnFamily;
+import static com.facebook.presto.accumulo.index.Indexer.getMetricsTableName;
+import static com.facebook.presto.spi.StandardErrorCode.FUNCTION_IMPLEMENTATION_ERROR;
+import static com.google.common.base.MoreObjects.toStringHelper;
+import static com.google.common.collect.Streams.stream;
+import static io.airlift.concurrent.Threads.daemonThreadsNamed;
+import static java.lang.Long.parseLong;
import static java.nio.charset.StandardCharsets.UTF_8;
import static java.util.Objects.requireNonNull;
+import static java.util.concurrent.Executors.newCachedThreadPool;
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
/**
* This class is an indexing utility to cache the cardinality of a column value for every table.
@@ -57,40 +81,34 @@
*/
public class ColumnCardinalityCache
{
- private final Authorizations auths;
private static final Logger LOG = Logger.get(ColumnCardinalityCache.class);
private final Connector connector;
- private final int size;
- private final Duration expireDuration;
+ private final ExecutorService coreExecutor;
+ private final BoundedExecutor executorService;
+ private final LoadingCache cache;
- @GuardedBy("this")
- private final Map tableToCache = new HashMap<>();
-
- public ColumnCardinalityCache(
- Connector connector,
- AccumuloConfig config,
- Authorizations auths)
+ @Inject
+ public ColumnCardinalityCache(Connector connector, AccumuloConfig config)
{
this.connector = requireNonNull(connector, "connector is null");
- this.size = requireNonNull(config, "config is null").getCardinalityCacheSize();
- this.expireDuration = config.getCardinalityCacheExpiration();
- this.auths = requireNonNull(auths, "auths is null");
+ int size = requireNonNull(config, "config is null").getCardinalityCacheSize();
+ Duration expireDuration = config.getCardinalityCacheExpiration();
+
+ // Create a bounded executor with a pool size at 4x number of processors
+ this.coreExecutor = newCachedThreadPool(daemonThreadsNamed("cardinality-lookup-%s"));
+ this.executorService = new BoundedExecutor(coreExecutor, 4 * Runtime.getRuntime().availableProcessors());
+
+ LOG.debug("Created new cache size %d expiry %s", size, expireDuration);
+ cache = CacheBuilder.newBuilder()
+ .maximumSize(size)
+ .expireAfterWrite(expireDuration.toMillis(), MILLISECONDS)
+ .build(new CardinalityCacheLoader());
}
- /**
- * Deletes any cache for the given table, no-op of table does not exist in the cache
- *
- * @param schema Schema name
- * @param table Table name
- */
- public synchronized void deleteCache(String schema, String table)
+ @PreDestroy
+ public void shutdown()
{
- LOG.debug("Deleting cache for %s.%s", schema, table);
- if (tableToCache.containsKey(table)) {
- // clear the cache and remove it
- getTableCache(schema, table).clear();
- tableToCache.remove(table);
- }
+ coreExecutor.shutdownNow();
}
/**
@@ -99,161 +117,210 @@ public synchronized void deleteCache(String schema, String table)
*
* @param schema Schema name
* @param table Table name
+ * @param auths Scan authorizations
* @param idxConstraintRangePairs Mapping of all ranges for a given constraint
+ * @param earlyReturnThreshold Smallest acceptable cardinality to return early while other tasks complete
+ * @param pollingDuration Duration for polling the cardinality completion service
* @return An immutable multimap of cardinality to column constraint, sorted by cardinality from smallest to largest
* @throws TableNotFoundException If the metrics table does not exist
* @throws ExecutionException If another error occurs; I really don't even know anymore.
*/
- public Multimap getCardinalities(String schema, String table, Multimap idxConstraintRangePairs)
+ public Multimap getCardinalities(String schema, String table, Authorizations auths, Multimap idxConstraintRangePairs, long earlyReturnThreshold, Duration pollingDuration)
throws ExecutionException, TableNotFoundException
{
- // Create a multi map sorted by cardinality, sort columns by name
- TreeMultimap cardinalityToConstraints = TreeMultimap.create(
- Long::compare,
- (AccumuloColumnConstraint o1, AccumuloColumnConstraint o2) -> o1.getName().compareTo(o2.getName()));
-
- for (Entry> entry : idxConstraintRangePairs.asMap().entrySet()) {
- long card = getColumnCardinality(schema, table, entry.getKey(), entry.getValue());
- LOG.debug("Cardinality for column %s is %s", entry.getKey().getName(), card);
- cardinalityToConstraints.put(card, entry.getKey());
+ // Submit tasks to the executor to fetch column cardinality, adding it to the Guava cache if necessary
+ CompletionService> executor = new ExecutorCompletionService<>(executorService);
+ idxConstraintRangePairs.asMap().forEach((key, value) -> executor.submit(() -> {
+ long cardinality = getColumnCardinality(schema, table, auths, key.getFamily(), key.getQualifier(), value);
+ LOG.debug("Cardinality for column %s is %s", key.getName(), cardinality);
+ return Pair.of(cardinality, key);
+ }
+ ));
+
+ // Create a multi map sorted by cardinality
+ ListMultimap cardinalityToConstraints = MultimapBuilder.treeKeys().arrayListValues().build();
+ try {
+ boolean earlyReturn = false;
+ int numTasks = idxConstraintRangePairs.asMap().entrySet().size();
+ do {
+ // Sleep for the polling duration to allow concurrent tasks to run for this time
+ Thread.sleep(pollingDuration.toMillis());
+
+ // Poll each task, retrieving the result if it is done
+ for (int i = 0; i < numTasks; ++i) {
+ Future> futureCardinality = executor.poll();
+ if (futureCardinality != null && futureCardinality.isDone()) {
+ Pair columnCardinality = futureCardinality.get();
+ cardinalityToConstraints.put(columnCardinality.getLeft(), columnCardinality.getRight());
+ }
+ }
+
+ // If the smallest cardinality is present and below the threshold, set the earlyReturn flag
+ Optional> smallestCardinality = cardinalityToConstraints.entries().stream().findFirst();
+ if (smallestCardinality.isPresent()) {
+ if (smallestCardinality.get().getKey() <= earlyReturnThreshold) {
+ LOG.info("Cardinality %s, is below threshold. Returning early while other tasks finish", smallestCardinality);
+ earlyReturn = true;
+ }
+ }
+ }
+ while (!earlyReturn && cardinalityToConstraints.entries().size() < numTasks);
+ }
+ catch (ExecutionException | InterruptedException e) {
+ if (e instanceof InterruptedException) {
+ Thread.currentThread().interrupt();
+ }
+ throw new PrestoException(UNEXPECTED_ACCUMULO_ERROR, "Exception when getting cardinality", e);
}
+ // Create a copy of the cardinalities
return ImmutableMultimap.copyOf(cardinalityToConstraints);
}
/**
- * Gets the cardinality for the given column constraint with the given Ranges.
- * Ranges can be exact values or a range of values.
+ * Gets the column cardinality for all of the given range values. May reach out to the
+ * metrics table in Accumulo to retrieve new cache elements.
*
- * @param schema Schema name
+ * @param schema Table schema
* @param table Table name
- * @param columnConstraint Mapping of all ranges for a given constraint
- * @param indexRanges Ranges for each exact or ranged value of the column constraint
- * @return The cardinality for the column
- * @throws TableNotFoundException If the metrics table does not exist
- * @throws ExecutionException If another error occurs; I really don't even know anymore.
+ * @param auths Scan authorizations
+ * @param family Accumulo column family
+ * @param qualifier Accumulo column qualifier
+ * @param colValues All range values to summarize for the cardinality
+ * @return The cardinality of the column
*/
- private long getColumnCardinality(String schema, String table, AccumuloColumnConstraint columnConstraint, Collection indexRanges)
- throws ExecutionException, TableNotFoundException
+ public long getColumnCardinality(String schema, String table, Authorizations auths, String family, String qualifier, Collection colValues)
+ throws ExecutionException
{
- return getTableCache(schema, table)
- .getColumnCardinality(
- columnConstraint.getName(),
- columnConstraint.getFamily(),
- columnConstraint.getQualifier(),
- indexRanges);
+ LOG.debug("Getting cardinality for %s:%s", family, qualifier);
+
+ // Collect all exact Accumulo Ranges, i.e. single value entries vs. a full scan
+ Collection exactRanges = colValues.stream()
+ .filter(ColumnCardinalityCache::isExact)
+ .map(range -> new CacheKey(schema, table, family, qualifier, range, auths))
+ .collect(Collectors.toList());
+
+ LOG.debug("Column values contain %s exact ranges of %s", exactRanges.size(), colValues.size());
+
+ // Sum the cardinalities for the exact-value Ranges
+ // This is where the reach-out to Accumulo occurs for all Ranges that have not
+ // previously been fetched
+ long sum = cache.getAll(exactRanges).values().stream().mapToLong(Long::longValue).sum();
+
+ // If these collection sizes are not equal,
+ // then there is at least one non-exact range
+ if (exactRanges.size() != colValues.size()) {
+ // for each range in the column value
+ for (Range range : colValues) {
+ // if this range is not exact
+ if (!isExact(range)) {
+ // Then get the value for this range using the single-value cache lookup
+ sum += cache.get(new CacheKey(schema, table, family, qualifier, range, auths));
+ }
+ }
+ }
+
+ return sum;
}
- /**
- * Gets the {@link TableColumnCache} for the given table, creating a new one if necessary.
- *
- * @param schema Schema name
- * @param table Table name
- * @return An existing or new TableColumnCache
- */
- private synchronized TableColumnCache getTableCache(String schema, String table)
+ private static boolean isExact(Range range)
{
- String fullName = AccumuloTable.getFullTableName(schema, table);
- TableColumnCache cache = tableToCache.get(fullName);
- if (cache == null) {
- LOG.debug("Creating new TableColumnCache for %s.%s %s", schema, table, this);
- cache = new TableColumnCache(schema, table);
- tableToCache.put(fullName, cache);
- }
- return cache;
+ return !range.isInfiniteStartKey() && !range.isInfiniteStopKey() &&
+ range.getStartKey().followingKey(PartialKey.ROW).equals(range.getEndKey());
}
/**
- * Internal class for holding the mapping of column names to the LoadingCache
+ * Complex key for the CacheLoader
*/
- private class TableColumnCache
+ private static class CacheKey
{
- private final Map> columnToCache = new HashMap<>();
private final String schema;
private final String table;
+ private final String family;
+ private final String qualifier;
+ private final Range range;
+ private final Authorizations auths;
- public TableColumnCache(String schema,
- String table)
+ public CacheKey(
+ String schema,
+ String table,
+ String family,
+ String qualifier,
+ Range range,
+ Authorizations auths)
{
- this.schema = schema;
- this.table = table;
+ this.schema = requireNonNull(schema, "schema is null");
+ this.table = requireNonNull(table, "table is null");
+ this.family = requireNonNull(family, "family is null");
+ this.qualifier = requireNonNull(qualifier, "qualifier is null");
+ this.range = requireNonNull(range, "range is null");
+ this.auths = requireNonNull(auths, "auths is null");
}
- /**
- * Clears and removes all caches as if the object had been first created
- */
- public void clear()
+ public String getSchema()
{
- columnToCache.values().forEach(LoadingCache::invalidateAll);
- columnToCache.clear();
+ return schema;
}
- /**
- * Gets the column cardinality for all of the given range values.
- * May reach out to the metrics table in Accumulo to retrieve new cache elements.
- *
- * @param column Presto column name
- * @param family Accumulo column family
- * @param qualifier Accumulo column qualifier
- * @param colValues All range values to summarize for the cardinality
- * @return The cardinality of the column
- */
- public long getColumnCardinality(String column, String family, String qualifier, Collection colValues)
- throws ExecutionException, TableNotFoundException
+ public String getTable()
{
- // Get the column cache for this column, creating a new one if necessary
- LoadingCache cache = columnToCache.get(column);
- if (cache == null) {
- cache = newCache(schema, table, family, qualifier);
- columnToCache.put(column, cache);
- }
+ return table;
+ }
- // Collect all exact Accumulo Ranges, i.e. single value entries vs. a full scan
- Collection exactRanges = colValues.stream().filter(this::isExact).collect(Collectors.toList());
- LOG.debug("Column values contain %s exact ranges of %s", exactRanges.size(), colValues.size());
+ public String getFamily()
+ {
+ return family;
+ }
- // Sum the cardinalities for the exact-value Ranges
- // This is where the reach-out to Accumulo occurs for all Ranges that have not previously been fetched
- long sum = 0;
- for (Long value : cache.getAll(exactRanges).values()) {
- sum += value;
- }
+ public String getQualifier()
+ {
+ return qualifier;
+ }
- // If these collection sizes are not equal, then there is at least one non-exact range
- if (exactRanges.size() != colValues.size()) {
- // for each range in the column value
- for (Range range : colValues) {
- // if this range is not exact
- if (!isExact(range)) {
- // Then get the value for this range using the single-value cache lookup
- long value = cache.get(range);
-
- // add our value to the cache and our sum
- cache.put(range, value);
- sum += value;
- }
- }
- }
+ public Range getRange()
+ {
+ return range;
+ }
- LOG.debug("Cache stats : size=%s, %s", cache.size(), cache.stats());
- return sum;
+ public Authorizations getAuths()
+ {
+ return auths;
}
- private boolean isExact(Range range)
+ @Override
+ public int hashCode()
{
- return !range.isInfiniteStartKey()
- && !range.isInfiniteStopKey()
- && range.getStartKey().followingKey(PartialKey.ROW).equals(range.getEndKey());
+ return Objects.hash(schema, table, family, qualifier, range);
}
- private LoadingCache newCache(String schema, String table, String family, String qualifier)
+ @Override
+ public boolean equals(Object obj)
{
- LOG.debug("Created new cache for %s.%s, column %s:%s, size %s expiry %s", schema, table, family, qualifier, size, expireDuration);
- return CacheBuilder
- .newBuilder()
- .maximumSize(size)
- .expireAfterWrite(expireDuration.toMillis(), TimeUnit.MILLISECONDS)
- .build(new CardinalityCacheLoader(schema, table, family, qualifier));
+ if (this == obj) {
+ return true;
+ }
+
+ if ((obj == null) || (getClass() != obj.getClass())) {
+ return false;
+ }
+
+ CacheKey other = (CacheKey) obj;
+ return Objects.equals(this.schema, other.schema)
+ && Objects.equals(this.table, other.table)
+ && Objects.equals(this.family, other.family)
+ && Objects.equals(this.qualifier, other.qualifier)
+ && Objects.equals(this.range, other.range);
+ }
+
+ @Override
+ public String toString()
+ {
+ return toStringHelper(this)
+ .add("schema", schema)
+ .add("table", table)
+ .add("family", family)
+ .add("qualifier", qualifier)
+ .add("range", range).toString();
}
}
@@ -261,23 +328,8 @@ private LoadingCache newCache(String schema, String table, String f
* Internal class for loading the cardinality from Accumulo
*/
private class CardinalityCacheLoader
- extends CacheLoader
+ extends CacheLoader
{
- private final String metricsTable;
- private final Text columnFamily;
-
- public CardinalityCacheLoader(
- String schema,
- String table,
- String family,
- String qualifier)
- {
- this.metricsTable = Indexer.getMetricsTableName(schema, table);
-
- // Create the column family for our scanners
- this.columnFamily = new Text(Indexer.getIndexColumnFamily(family.getBytes(UTF_8), qualifier.getBytes(UTF_8)).array());
- }
-
/**
* Loads the cardinality for the given Range. Uses a BatchScanner and sums the cardinality for all values that encapsulate the Range.
*
@@ -285,64 +337,77 @@ public CardinalityCacheLoader(
* @return The cardinality of the column, which would be zero if the value does not exist
*/
@Override
- public Long load(Range key)
+ public Long load(@Nonnull CacheKey key)
throws Exception
{
- // Create a BatchScanner against our metrics table, setting the value range and fetching the appropriate column
- BatchScanner scanner = connector.createBatchScanner(metricsTable, auths, 10);
- scanner.setRanges(ImmutableList.of(key));
- scanner.fetchColumn(columnFamily, Indexer.CARDINALITY_CQ_AS_TEXT);
-
- // Sum all those entries!
- long sum = 0;
- for (Entry entry : scanner) {
- sum += Long.parseLong(entry.getValue().toString());
+ LOG.debug("Loading a non-exact range from Accumulo: %s", key);
+ // Get metrics table name and the column family for the scanner
+ String metricsTable = getMetricsTableName(key.getSchema(), key.getTable());
+ Text columnFamily = new Text(getIndexColumnFamily(key.getFamily().getBytes(UTF_8), key.getQualifier().getBytes(UTF_8)).array());
+
+ // Create scanner for querying the range
+ Scanner scanner = connector.createScanner(metricsTable, key.getAuths());
+ scanner.setRange(key.getRange());
+ scanner.fetchColumn(columnFamily, CARDINALITY_CQ_AS_TEXT);
+
+ try {
+ return stream(scanner)
+ .map(Entry::getValue)
+ .map(Value::toString)
+ .mapToLong(Long::parseLong)
+ .sum();
+ }
+ finally {
+ scanner.close();
}
-
- scanner.close();
- return sum;
}
- @SuppressWarnings("unchecked")
@Override
- public Map loadAll(Iterable extends Range> keys)
+ public Map loadAll(@Nonnull Iterable extends CacheKey> keys)
throws Exception
{
- LOG.debug("Loading %s exact ranges from Accumulo", ((Collection) keys).size());
-
- // Create batch scanner for querying all ranges
- BatchScanner scanner = connector.createBatchScanner(metricsTable, auths, 10);
- scanner.setRanges((Collection) keys);
- scanner.fetchColumn(columnFamily, Indexer.CARDINALITY_CQ_AS_TEXT);
-
- // Create a new map to hold our cardinalities for each range, returning a default of zero for each non-existent Key
- Map rangeValues = new MapDefaultZero();
- for (Entry entry : scanner) {
- rangeValues.put(
- Range.exact(entry.getKey().getRow()),
- Long.parseLong(entry.getValue().toString()));
+ int size = Iterables.size(keys);
+ if (size == 0) {
+ return ImmutableMap.of();
}
- scanner.close();
- return rangeValues;
- }
+ LOG.debug("Loading %s exact ranges from Accumulo", size);
- /**
- * We extend HashMap here and override get to return a value of zero if the key is not in the map.
- * This mitigates the CacheLoader InvalidCacheLoadException if loadAll fails to return a value for a given key,
- * which occurs when there is no key in Accumulo.
- */
- public class MapDefaultZero
- extends HashMap
- {
- @Override
- public Long get(Object key)
- {
- // Get the key from our map overlord
- Long value = super.get(key);
-
- // Return zero if null
- return value == null ? 0 : value;
+ // In order to simplify the implementation, we are making a (safe) assumption
+ // that the CacheKeys will all contain the same combination of schema/table/family/qualifier
+ // This is asserted with the below implementation error just to make sure
+ CacheKey anyKey = stream(keys).findAny().get();
+ if (stream(keys).anyMatch(k -> !k.getSchema().equals(anyKey.getSchema()) || !k.getTable().equals(anyKey.getTable()) || !k.getFamily().equals(anyKey.getFamily()) || !k.getQualifier().equals(anyKey.getQualifier()))) {
+ throw new PrestoException(FUNCTION_IMPLEMENTATION_ERROR, "loadAll called with a non-homogeneous collection of cache keys");
+ }
+
+ Map rangeToKey = stream(keys).collect(Collectors.toMap(CacheKey::getRange, Function.identity()));
+ LOG.debug("rangeToKey size is %s", rangeToKey.size());
+
+ // Get metrics table name and the column family for the scanner
+ String metricsTable = getMetricsTableName(anyKey.getSchema(), anyKey.getTable());
+ Text columnFamily = new Text(getIndexColumnFamily(anyKey.getFamily().getBytes(UTF_8), anyKey.getQualifier().getBytes(UTF_8)).array());
+
+ BatchScanner scanner = connector.createBatchScanner(metricsTable, anyKey.getAuths(), 10);
+ try {
+ scanner.setRanges(stream(keys).map(CacheKey::getRange).collect(Collectors.toList()));
+ scanner.fetchColumn(columnFamily, CARDINALITY_CQ_AS_TEXT);
+
+ // Create a new map to hold our cardinalities for each range, returning a default of
+ // Zero for each non-existent Key
+ Map rangeValues = new HashMap<>();
+ stream(keys).forEach(key -> rangeValues.put(key, 0L));
+
+ for (Entry entry : scanner) {
+ rangeValues.put(rangeToKey.get(Range.exact(entry.getKey().getRow())), parseLong(entry.getValue().toString()));
+ }
+
+ return rangeValues;
+ }
+ finally {
+ if (scanner != null) {
+ scanner.close();
+ }
}
}
}
diff --git a/presto-accumulo/src/main/java/com/facebook/presto/accumulo/index/IndexLookup.java b/presto-accumulo/src/main/java/com/facebook/presto/accumulo/index/IndexLookup.java
index e5c359f5bf166..8c0ee86ab2635 100644
--- a/presto-accumulo/src/main/java/com/facebook/presto/accumulo/index/IndexLookup.java
+++ b/presto-accumulo/src/main/java/com/facebook/presto/accumulo/index/IndexLookup.java
@@ -13,9 +13,6 @@
*/
package com.facebook.presto.accumulo.index;
-import com.facebook.presto.accumulo.AccumuloClient;
-import com.facebook.presto.accumulo.conf.AccumuloConfig;
-import com.facebook.presto.accumulo.conf.AccumuloSessionProperties;
import com.facebook.presto.accumulo.model.AccumuloColumnConstraint;
import com.facebook.presto.accumulo.model.TabletSplitMetadata;
import com.facebook.presto.accumulo.serializers.AccumuloRowSerializer;
@@ -26,6 +23,7 @@
import com.google.common.collect.ImmutableMultimap;
import com.google.common.collect.Multimap;
import io.airlift.log.Logger;
+import io.airlift.units.Duration;
import org.apache.accumulo.core.client.AccumuloException;
import org.apache.accumulo.core.client.AccumuloSecurityException;
import org.apache.accumulo.core.client.BatchScanner;
@@ -38,13 +36,30 @@
import org.apache.accumulo.core.security.Authorizations;
import org.apache.hadoop.io.Text;
+import javax.inject.Inject;
+
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Map.Entry;
import java.util.Optional;
import java.util.Set;
-
+import java.util.concurrent.TimeUnit;
+
+import static com.facebook.presto.accumulo.AccumuloClient.getRangesFromDomain;
+import static com.facebook.presto.accumulo.conf.AccumuloSessionProperties.getIndexCardinalityCachePollingDuration;
+import static com.facebook.presto.accumulo.conf.AccumuloSessionProperties.getIndexSmallCardThreshold;
+import static com.facebook.presto.accumulo.conf.AccumuloSessionProperties.getIndexThreshold;
+import static com.facebook.presto.accumulo.conf.AccumuloSessionProperties.getNumIndexRowsPerSplit;
+import static com.facebook.presto.accumulo.conf.AccumuloSessionProperties.isIndexMetricsEnabled;
+import static com.facebook.presto.accumulo.conf.AccumuloSessionProperties.isIndexShortCircuitEnabled;
+import static com.facebook.presto.accumulo.conf.AccumuloSessionProperties.isOptimizeIndexEnabled;
+import static com.facebook.presto.accumulo.index.Indexer.CARDINALITY_CQ_AS_TEXT;
+import static com.facebook.presto.accumulo.index.Indexer.METRICS_TABLE_ROWID_AS_TEXT;
+import static com.facebook.presto.accumulo.index.Indexer.METRICS_TABLE_ROWS_CF_AS_TEXT;
+import static com.facebook.presto.accumulo.index.Indexer.getIndexColumnFamily;
+import static com.facebook.presto.accumulo.index.Indexer.getIndexTableName;
+import static com.facebook.presto.accumulo.index.Indexer.getMetricsTableName;
import static com.facebook.presto.spi.StandardErrorCode.FUNCTION_IMPLEMENTATION_ERROR;
import static com.google.common.base.Preconditions.checkArgument;
import static java.nio.charset.StandardCharsets.UTF_8;
@@ -59,22 +74,15 @@
public class IndexLookup
{
private static final Logger LOG = Logger.get(IndexLookup.class);
- private static final Range METRICS_TABLE_ROWID_RANGE = new Range(Indexer.METRICS_TABLE_ROWID_AS_TEXT);
+ private static final Range METRICS_TABLE_ROWID_RANGE = new Range(METRICS_TABLE_ROWID_AS_TEXT);
private final ColumnCardinalityCache cardinalityCache;
private final Connector connector;
- public IndexLookup(
- Connector connector,
- AccumuloConfig config,
- Authorizations auths)
+ @Inject
+ public IndexLookup(Connector connector, ColumnCardinalityCache cardinalityCache)
{
this.connector = requireNonNull(connector, "connector is null");
- this.cardinalityCache = new ColumnCardinalityCache(connector, requireNonNull(config, "config is null"), auths);
- }
-
- public void dropCache(String schema, String table)
- {
- cardinalityCache.deleteCache(schema, table);
+ this.cardinalityCache = requireNonNull(cardinalityCache, "cardinalityCache is null");
}
/**
@@ -111,7 +119,7 @@ public boolean applyIndex(
throws Exception
{
// Early out if index is disabled
- if (!AccumuloSessionProperties.isOptimizeIndexEnabled(session)) {
+ if (!isOptimizeIndexEnabled(session)) {
LOG.debug("Secondary index is disabled");
return false;
}
@@ -128,14 +136,14 @@ public boolean applyIndex(
}
// If metrics are not enabled
- if (!AccumuloSessionProperties.isIndexMetricsEnabled(session)) {
+ if (!isIndexMetricsEnabled(session)) {
LOG.debug("Use of index metrics is disabled");
// Get the ranges via the index table
- List indexRanges = getIndexRanges(Indexer.getIndexTableName(schema, table), constraintRanges, rowIdRanges, auths);
+ List indexRanges = getIndexRanges(getIndexTableName(schema, table), constraintRanges, rowIdRanges, auths);
if (!indexRanges.isEmpty()) {
// Bin the ranges into TabletMetadataSplits and return true to use the tablet splits
- binRanges(AccumuloSessionProperties.getNumIndexRowsPerSplit(session), indexRanges, tabletSplits);
+ binRanges(getNumIndexRowsPerSplit(session), indexRanges, tabletSplits);
LOG.debug("Number of splits for %s.%s is %d with %d ranges", schema, table, tabletSplits.size(), indexRanges.size());
}
else {
@@ -157,14 +165,12 @@ private static Multimap getIndexedConstraintRan
ImmutableListMultimap.Builder builder = ImmutableListMultimap.builder();
for (AccumuloColumnConstraint columnConstraint : constraints) {
if (columnConstraint.isIndexed()) {
- for (Range range : AccumuloClient.getRangesFromDomain(columnConstraint.getDomain(), serializer)) {
+ for (Range range : getRangesFromDomain(columnConstraint.getDomain(), serializer)) {
builder.put(columnConstraint, range);
}
}
else {
- LOG.warn(
- "Query containts constraint on non-indexed column %s. Is it worth indexing?",
- columnConstraint.getName());
+ LOG.warn("Query containts constraint on non-indexed column %s. Is it worth indexing?", columnConstraint.getName());
}
}
return builder.build();
@@ -180,18 +186,33 @@ private boolean getRangesWithMetrics(
Authorizations auths)
throws Exception
{
+ String metricsTable = getMetricsTableName(schema, table);
+ long numRows = getNumRowsInTable(metricsTable, auths);
+
// Get the cardinalities from the metrics table
- Multimap cardinalities = cardinalityCache.getCardinalities(schema, table, constraintRanges);
+ Multimap cardinalities;
+ if (isIndexShortCircuitEnabled(session)) {
+ cardinalities = cardinalityCache.getCardinalities(
+ schema,
+ table,
+ auths,
+ constraintRanges,
+ (long) (numRows * getIndexSmallCardThreshold(session)),
+ getIndexCardinalityCachePollingDuration(session));
+ }
+ else {
+ // disable short circuit using 0
+ cardinalities = cardinalityCache.getCardinalities(schema, table, auths, constraintRanges, 0, new Duration(0, TimeUnit.MILLISECONDS));
+ }
+
Optional> entry = cardinalities.entries().stream().findFirst();
if (!entry.isPresent()) {
return false;
}
Entry lowestCardinality = entry.get();
- String indexTable = Indexer.getIndexTableName(schema, table);
- String metricsTable = Indexer.getMetricsTableName(schema, table);
- long numRows = getNumRowsInTable(metricsTable, auths);
- double threshold = AccumuloSessionProperties.getIndexThreshold(session);
+ String indexTable = getIndexTableName(schema, table);
+ double threshold = getIndexThreshold(session);
List indexRanges;
// If the smallest cardinality in our list is above the lowest cardinality threshold,
@@ -235,7 +256,7 @@ private boolean getRangesWithMetrics(
// If the percentage of scanned rows, the ratio, less than the configured threshold
if (ratio < threshold) {
// Bin the ranges into TabletMetadataSplits and return true to use the tablet splits
- binRanges(AccumuloSessionProperties.getNumIndexRowsPerSplit(session), indexRanges, tabletSplits);
+ binRanges(getNumIndexRowsPerSplit(session), indexRanges, tabletSplits);
LOG.debug("Number of splits for %s.%s is %d with %d ranges", schema, table, tabletSplits.size(), indexRanges.size());
return true;
}
@@ -248,7 +269,7 @@ private boolean getRangesWithMetrics(
private static boolean smallestCardAboveThreshold(ConnectorSession session, long numRows, long smallestCardinality)
{
double ratio = ((double) smallestCardinality / (double) numRows);
- double threshold = AccumuloSessionProperties.getIndexSmallCardThreshold(session);
+ double threshold = getIndexSmallCardThreshold(session);
LOG.debug("Smallest cardinality is %d, num rows is %d, ratio is %2f with threshold of %f", smallestCardinality, numRows, ratio, threshold);
return ratio > threshold;
}
@@ -259,7 +280,7 @@ private long getNumRowsInTable(String metricsTable, Authorizations auths)
// Create scanner against the metrics table, pulling the special column and the rows column
Scanner scanner = connector.createScanner(metricsTable, auths);
scanner.setRange(METRICS_TABLE_ROWID_RANGE);
- scanner.fetchColumn(Indexer.METRICS_TABLE_ROWS_CF_AS_TEXT, Indexer.CARDINALITY_CQ_AS_TEXT);
+ scanner.fetchColumn(METRICS_TABLE_ROWS_CF_AS_TEXT, CARDINALITY_CQ_AS_TEXT);
// Scan the entry and get the number of rows
long numRows = -1;
@@ -286,10 +307,7 @@ private List getIndexRanges(String indexTable, Multimap map)
Type keyType = mapType.getTypeParameters().get(0);
Type valueType = mapType.getTypeParameters().get(1);
- BlockBuilder builder = new InterleavedBlockBuilder(ImmutableList.of(keyType, valueType), new BlockBuilderStatus(), map.size() * 2);
+ BlockBuilder mapBlockBuilder = mapType.createBlockBuilder(new BlockBuilderStatus(), 1);
+ BlockBuilder builder = mapBlockBuilder.beginBlockEntry();
for (Entry, ?> entry : map.entrySet()) {
writeObject(builder, keyType, entry.getKey());
writeObject(builder, valueType, entry.getValue());
}
- return builder.build();
+
+ mapBlockBuilder.closeEntry();
+ return (Block) mapType.getObject(mapBlockBuilder, 0);
}
/**
diff --git a/presto-accumulo/src/test/java/com/facebook/presto/accumulo/TestAccumuloClient.java b/presto-accumulo/src/test/java/com/facebook/presto/accumulo/TestAccumuloClient.java
index 2d4da61560ba4..21d86ecce4253 100644
--- a/presto-accumulo/src/test/java/com/facebook/presto/accumulo/TestAccumuloClient.java
+++ b/presto-accumulo/src/test/java/com/facebook/presto/accumulo/TestAccumuloClient.java
@@ -15,6 +15,8 @@
import com.facebook.presto.accumulo.conf.AccumuloConfig;
import com.facebook.presto.accumulo.conf.AccumuloTableProperties;
+import com.facebook.presto.accumulo.index.ColumnCardinalityCache;
+import com.facebook.presto.accumulo.index.IndexLookup;
import com.facebook.presto.accumulo.metadata.AccumuloTable;
import com.facebook.presto.accumulo.metadata.ZooKeeperMetadataManager;
import com.facebook.presto.spi.ColumnMetadata;
@@ -47,7 +49,7 @@ public TestAccumuloClient()
Connector connector = AccumuloQueryRunner.getAccumuloConnector();
config.setZooKeepers(connector.getInstance().getZooKeepers());
zooKeeperMetadataManager = new ZooKeeperMetadataManager(config, new TypeRegistry());
- client = new AccumuloClient(connector, config, zooKeeperMetadataManager, new AccumuloTableManager(connector));
+ client = new AccumuloClient(connector, config, zooKeeperMetadataManager, new AccumuloTableManager(connector), new IndexLookup(connector, new ColumnCardinalityCache(connector, config)));
}
@Test
diff --git a/presto-accumulo/src/test/java/com/facebook/presto/accumulo/index/TestIndexer.java b/presto-accumulo/src/test/java/com/facebook/presto/accumulo/index/TestIndexer.java
index 6160553aac845..1282ca71ef434 100644
--- a/presto-accumulo/src/test/java/com/facebook/presto/accumulo/index/TestIndexer.java
+++ b/presto-accumulo/src/test/java/com/facebook/presto/accumulo/index/TestIndexer.java
@@ -17,8 +17,8 @@
import com.facebook.presto.accumulo.model.AccumuloColumnHandle;
import com.facebook.presto.accumulo.serializers.AccumuloRowSerializer;
import com.facebook.presto.accumulo.serializers.LexicoderRowSerializer;
+import com.facebook.presto.spi.type.ArrayType;
import com.facebook.presto.spi.type.Type;
-import com.facebook.presto.type.ArrayType;
import com.google.common.collect.ImmutableList;
import org.apache.accumulo.core.client.BatchWriterConfig;
import org.apache.accumulo.core.client.Connector;
diff --git a/presto-accumulo/src/test/java/com/facebook/presto/accumulo/model/TestField.java b/presto-accumulo/src/test/java/com/facebook/presto/accumulo/model/TestField.java
index 6ae3492f9598c..55db1b1e98e27 100644
--- a/presto-accumulo/src/test/java/com/facebook/presto/accumulo/model/TestField.java
+++ b/presto-accumulo/src/test/java/com/facebook/presto/accumulo/model/TestField.java
@@ -14,10 +14,16 @@
package com.facebook.presto.accumulo.model;
import com.facebook.presto.accumulo.serializers.AccumuloRowSerializer;
+import com.facebook.presto.block.BlockEncodingManager;
+import com.facebook.presto.metadata.FunctionRegistry;
import com.facebook.presto.spi.block.Block;
+import com.facebook.presto.spi.type.ArrayType;
+import com.facebook.presto.spi.type.StandardTypes;
import com.facebook.presto.spi.type.Type;
-import com.facebook.presto.type.ArrayType;
-import com.facebook.presto.type.MapType;
+import com.facebook.presto.spi.type.TypeManager;
+import com.facebook.presto.spi.type.TypeSignatureParameter;
+import com.facebook.presto.sql.analyzer.FeaturesConfig;
+import com.facebook.presto.type.TypeRegistry;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import org.testng.annotations.Test;
@@ -172,7 +178,13 @@ public void testLong()
public void testMap()
throws Exception
{
- Type type = new MapType(VARCHAR, BIGINT);
+ TypeManager typeManager = new TypeRegistry();
+ // associate typeManager with a function registry
+ new FunctionRegistry(typeManager, new BlockEncodingManager(typeManager), new FeaturesConfig());
+
+ Type type = typeManager.getParameterizedType(StandardTypes.MAP, ImmutableList.of(
+ TypeSignatureParameter.of(VARCHAR.getTypeSignature()),
+ TypeSignatureParameter.of(BIGINT.getTypeSignature())));
Block expected = AccumuloRowSerializer.getBlockFromMap(type, ImmutableMap.of("a", 1L, "b", 2L, "c", 3L));
Field f1 = new Field(expected, type);
assertEquals(f1.getMap(), expected);
diff --git a/presto-accumulo/src/test/java/com/facebook/presto/accumulo/model/TestRow.java b/presto-accumulo/src/test/java/com/facebook/presto/accumulo/model/TestRow.java
index 463cb01b19db6..596c81b0c4ea2 100644
--- a/presto-accumulo/src/test/java/com/facebook/presto/accumulo/model/TestRow.java
+++ b/presto-accumulo/src/test/java/com/facebook/presto/accumulo/model/TestRow.java
@@ -14,7 +14,7 @@
package com.facebook.presto.accumulo.model;
import com.facebook.presto.accumulo.serializers.AccumuloRowSerializer;
-import com.facebook.presto.type.ArrayType;
+import com.facebook.presto.spi.type.ArrayType;
import com.google.common.collect.ImmutableList;
import org.testng.annotations.Test;
@@ -22,7 +22,6 @@
import java.sql.Timestamp;
import java.util.GregorianCalendar;
import java.util.Optional;
-import java.util.concurrent.TimeUnit;
import static com.facebook.presto.spi.type.BigintType.BIGINT;
import static com.facebook.presto.spi.type.BooleanType.BOOLEAN;
@@ -83,7 +82,7 @@ public void testRowFromString()
Row expected = new Row();
expected.addField(new Field(AccumuloRowSerializer.getBlockFromArray(VARCHAR, ImmutableList.of("a", "b", "c")), new ArrayType(VARCHAR)));
expected.addField(true, BOOLEAN);
- expected.addField(new Field(new Date(TimeUnit.MILLISECONDS.toDays(new GregorianCalendar(1999, 0, 1).getTime().getTime())), DATE));
+ expected.addField(new Field(new Date(new GregorianCalendar(1999, 0, 1).getTime().getTime()), DATE));
expected.addField(123.45678, DOUBLE);
expected.addField(new Field(123.45678f, REAL));
expected.addField(12345678, INTEGER);
diff --git a/presto-accumulo/src/test/java/com/facebook/presto/accumulo/serializers/AbstractTestAccumuloRowSerializer.java b/presto-accumulo/src/test/java/com/facebook/presto/accumulo/serializers/AbstractTestAccumuloRowSerializer.java
index 47d6cd153a797..dec82d6ac2194 100644
--- a/presto-accumulo/src/test/java/com/facebook/presto/accumulo/serializers/AbstractTestAccumuloRowSerializer.java
+++ b/presto-accumulo/src/test/java/com/facebook/presto/accumulo/serializers/AbstractTestAccumuloRowSerializer.java
@@ -13,9 +13,15 @@
*/
package com.facebook.presto.accumulo.serializers;
+import com.facebook.presto.block.BlockEncodingManager;
+import com.facebook.presto.metadata.FunctionRegistry;
+import com.facebook.presto.spi.type.ArrayType;
+import com.facebook.presto.spi.type.StandardTypes;
import com.facebook.presto.spi.type.Type;
-import com.facebook.presto.type.ArrayType;
-import com.facebook.presto.type.MapType;
+import com.facebook.presto.spi.type.TypeManager;
+import com.facebook.presto.spi.type.TypeSignatureParameter;
+import com.facebook.presto.sql.analyzer.FeaturesConfig;
+import com.facebook.presto.type.TypeRegistry;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import org.apache.accumulo.core.data.Key;
@@ -182,8 +188,14 @@ public void testLong()
public void testMap()
throws Exception
{
+ TypeManager typeManager = new TypeRegistry();
+ // associate typeManager with a function registry
+ new FunctionRegistry(typeManager, new BlockEncodingManager(typeManager), new FeaturesConfig());
+
AccumuloRowSerializer serializer = serializerClass.getConstructor().newInstance();
- Type type = new MapType(VARCHAR, BIGINT);
+ Type type = typeManager.getParameterizedType(StandardTypes.MAP, ImmutableList.of(
+ TypeSignatureParameter.of(VARCHAR.getTypeSignature()),
+ TypeSignatureParameter.of(BIGINT.getTypeSignature())));
Map
+
+ it.unimi.dsi
+ fastutil
+
+
com.facebook.presto
presto-spi
@@ -30,5 +35,12 @@
org.openjdk.jol
jol-core
+
+
+
+ org.testng
+ testng
+ test
+
diff --git a/presto-array/src/main/java/com/facebook/presto/array/BlockBigArray.java b/presto-array/src/main/java/com/facebook/presto/array/BlockBigArray.java
index 24d75ea1d21e7..f6bb1106315b8 100644
--- a/presto-array/src/main/java/com/facebook/presto/array/BlockBigArray.java
+++ b/presto-array/src/main/java/com/facebook/presto/array/BlockBigArray.java
@@ -14,10 +14,13 @@
package com.facebook.presto.array;
import com.facebook.presto.spi.block.Block;
+import org.openjdk.jol.info.ClassLayout;
public final class BlockBigArray
{
+ private static final int INSTANCE_SIZE = ClassLayout.parseClass(BlockBigArray.class).instanceSize();
private final ObjectBigArray array;
+ private final ReferenceCountMap trackedObjects = new ReferenceCountMap();
private long sizeOfBlocks;
public BlockBigArray()
@@ -35,7 +38,7 @@ public BlockBigArray(Block block)
*/
public long sizeOf()
{
- return array.sizeOf() + sizeOfBlocks;
+ return INSTANCE_SIZE + array.sizeOf() + sizeOfBlocks + trackedObjects.sizeOf();
}
/**
@@ -58,10 +61,30 @@ public void set(long index, Block value)
{
Block currentValue = array.get(index);
if (currentValue != null) {
- sizeOfBlocks -= currentValue.getRetainedSizeInBytes();
+ currentValue.retainedBytesForEachPart((object, size) -> {
+ if (currentValue == object) {
+ // track instance size separately as the reference count for an instance is always 1
+ sizeOfBlocks -= size;
+ return;
+ }
+ if (trackedObjects.decrementReference(object) == 0) {
+ // decrement the size only when it is the last reference
+ sizeOfBlocks -= size;
+ }
+ });
}
if (value != null) {
- sizeOfBlocks += value.getRetainedSizeInBytes();
+ value.retainedBytesForEachPart((object, size) -> {
+ if (value == object) {
+ // track instance size separately as the reference count for an instance is always 1
+ sizeOfBlocks += size;
+ return;
+ }
+ if (trackedObjects.incrementReference(object) == 1) {
+ // increment the size only when it is the first reference
+ sizeOfBlocks += size;
+ }
+ });
}
array.set(index, value);
}
diff --git a/presto-array/src/main/java/com/facebook/presto/array/BooleanBigArray.java b/presto-array/src/main/java/com/facebook/presto/array/BooleanBigArray.java
index 0a85ac7e7be34..a46881784d22d 100644
--- a/presto-array/src/main/java/com/facebook/presto/array/BooleanBigArray.java
+++ b/presto-array/src/main/java/com/facebook/presto/array/BooleanBigArray.java
@@ -14,6 +14,7 @@
package com.facebook.presto.array;
import io.airlift.slice.SizeOf;
+import org.openjdk.jol.info.ClassLayout;
import java.util.Arrays;
@@ -27,6 +28,7 @@
// Copyright (C) 2010-2013 Sebastiano Vigna
public final class BooleanBigArray
{
+ private static final int INSTANCE_SIZE = ClassLayout.parseClass(BooleanBigArray.class).instanceSize();
private static final long SIZE_OF_SEGMENT = sizeOfBooleanArray(SEGMENT_SIZE);
private final boolean initialValue;
@@ -55,7 +57,7 @@ public BooleanBigArray(boolean initialValue)
*/
public long sizeOf()
{
- return SizeOf.sizeOf(array) + (segments * SIZE_OF_SEGMENT);
+ return INSTANCE_SIZE + SizeOf.sizeOf(array) + (segments * SIZE_OF_SEGMENT);
}
/**
diff --git a/presto-array/src/main/java/com/facebook/presto/array/ByteBigArray.java b/presto-array/src/main/java/com/facebook/presto/array/ByteBigArray.java
index 72a241855b80b..e9745ff10f2de 100644
--- a/presto-array/src/main/java/com/facebook/presto/array/ByteBigArray.java
+++ b/presto-array/src/main/java/com/facebook/presto/array/ByteBigArray.java
@@ -14,6 +14,7 @@
package com.facebook.presto.array;
import io.airlift.slice.SizeOf;
+import org.openjdk.jol.info.ClassLayout;
import java.util.Arrays;
@@ -27,6 +28,7 @@
// Copyright (C) 2010-2013 Sebastiano Vigna
public final class ByteBigArray
{
+ private static final int INSTANCE_SIZE = ClassLayout.parseClass(ByteBigArray.class).instanceSize();
private static final long SIZE_OF_SEGMENT = sizeOfByteArray(SEGMENT_SIZE);
private final byte initialValue;
@@ -55,7 +57,7 @@ public ByteBigArray(byte initialValue)
*/
public long sizeOf()
{
- return SizeOf.sizeOf(array) + (segments * SIZE_OF_SEGMENT);
+ return INSTANCE_SIZE + SizeOf.sizeOf(array) + (segments * SIZE_OF_SEGMENT);
}
/**
diff --git a/presto-array/src/main/java/com/facebook/presto/array/DoubleBigArray.java b/presto-array/src/main/java/com/facebook/presto/array/DoubleBigArray.java
index f0157d5f41d6b..a4b14db86cafb 100644
--- a/presto-array/src/main/java/com/facebook/presto/array/DoubleBigArray.java
+++ b/presto-array/src/main/java/com/facebook/presto/array/DoubleBigArray.java
@@ -14,6 +14,7 @@
package com.facebook.presto.array;
import io.airlift.slice.SizeOf;
+import org.openjdk.jol.info.ClassLayout;
import java.util.Arrays;
@@ -27,6 +28,7 @@
// Copyright (C) 2010-2013 Sebastiano Vigna
public final class DoubleBigArray
{
+ private static final int INSTANCE_SIZE = ClassLayout.parseClass(DoubleBigArray.class).instanceSize();
private static final long SIZE_OF_SEGMENT = sizeOfDoubleArray(SEGMENT_SIZE);
private final double initialValue;
@@ -58,7 +60,7 @@ public DoubleBigArray(double initialValue)
*/
public long sizeOf()
{
- return SizeOf.sizeOf(array) + (segments * SIZE_OF_SEGMENT);
+ return INSTANCE_SIZE + SizeOf.sizeOf(array) + (segments * SIZE_OF_SEGMENT);
}
/**
diff --git a/presto-array/src/main/java/com/facebook/presto/array/IntBigArray.java b/presto-array/src/main/java/com/facebook/presto/array/IntBigArray.java
index dc94b924ee956..deefed913c142 100644
--- a/presto-array/src/main/java/com/facebook/presto/array/IntBigArray.java
+++ b/presto-array/src/main/java/com/facebook/presto/array/IntBigArray.java
@@ -14,6 +14,7 @@
package com.facebook.presto.array;
import io.airlift.slice.SizeOf;
+import org.openjdk.jol.info.ClassLayout;
import java.util.Arrays;
@@ -21,13 +22,14 @@
import static com.facebook.presto.array.BigArrays.SEGMENT_SIZE;
import static com.facebook.presto.array.BigArrays.offset;
import static com.facebook.presto.array.BigArrays.segment;
-import static io.airlift.slice.SizeOf.sizeOfLongArray;
+import static io.airlift.slice.SizeOf.sizeOfIntArray;
// Note: this code was forked from fastutil (http://fastutil.di.unimi.it/)
// Copyright (C) 2010-2013 Sebastiano Vigna
public final class IntBigArray
{
- private static final long SIZE_OF_SEGMENT = sizeOfLongArray(SEGMENT_SIZE);
+ private static final int INSTANCE_SIZE = ClassLayout.parseClass(IntBigArray.class).instanceSize();
+ private static final long SIZE_OF_SEGMENT = sizeOfIntArray(SEGMENT_SIZE);
private final int initialValue;
@@ -58,7 +60,7 @@ public IntBigArray(int initialValue)
*/
public long sizeOf()
{
- return SizeOf.sizeOf(array) + (segments * SIZE_OF_SEGMENT);
+ return INSTANCE_SIZE + SizeOf.sizeOf(array) + (segments * SIZE_OF_SEGMENT);
}
/**
diff --git a/presto-array/src/main/java/com/facebook/presto/array/LongBigArray.java b/presto-array/src/main/java/com/facebook/presto/array/LongBigArray.java
index a55c3754ae02f..b5e49beedcfa7 100644
--- a/presto-array/src/main/java/com/facebook/presto/array/LongBigArray.java
+++ b/presto-array/src/main/java/com/facebook/presto/array/LongBigArray.java
@@ -14,6 +14,7 @@
package com.facebook.presto.array;
import io.airlift.slice.SizeOf;
+import org.openjdk.jol.info.ClassLayout;
import java.util.Arrays;
@@ -27,6 +28,7 @@
// Copyright (C) 2010-2013 Sebastiano Vigna
public final class LongBigArray
{
+ private static final int INSTANCE_SIZE = ClassLayout.parseClass(LongBigArray.class).instanceSize();
private static final long SIZE_OF_SEGMENT = sizeOfLongArray(SEGMENT_SIZE);
private final long initialValue;
@@ -58,7 +60,7 @@ public LongBigArray(long initialValue)
*/
public long sizeOf()
{
- return SizeOf.sizeOf(array) + (segments * SIZE_OF_SEGMENT);
+ return INSTANCE_SIZE + SizeOf.sizeOf(array) + (segments * SIZE_OF_SEGMENT);
}
/**
diff --git a/presto-array/src/main/java/com/facebook/presto/array/ObjectBigArray.java b/presto-array/src/main/java/com/facebook/presto/array/ObjectBigArray.java
index 156203c07f35c..c88d1a1eae890 100644
--- a/presto-array/src/main/java/com/facebook/presto/array/ObjectBigArray.java
+++ b/presto-array/src/main/java/com/facebook/presto/array/ObjectBigArray.java
@@ -14,6 +14,7 @@
package com.facebook.presto.array;
import io.airlift.slice.SizeOf;
+import org.openjdk.jol.info.ClassLayout;
import java.util.Arrays;
@@ -27,6 +28,7 @@
// Copyright (C) 2010-2013 Sebastiano Vigna
public final class ObjectBigArray
{
+ private static final int INSTANCE_SIZE = ClassLayout.parseClass(ObjectBigArray.class).instanceSize();
private static final long SIZE_OF_SEGMENT = sizeOfObjectArray(SEGMENT_SIZE);
private final Object initialValue;
@@ -55,7 +57,7 @@ public ObjectBigArray(Object initialValue)
*/
public long sizeOf()
{
- return SizeOf.sizeOf(array) + (segments * SIZE_OF_SEGMENT);
+ return INSTANCE_SIZE + SizeOf.sizeOf(array) + (segments * SIZE_OF_SEGMENT);
}
/**
diff --git a/presto-array/src/main/java/com/facebook/presto/array/ReferenceCountMap.java b/presto-array/src/main/java/com/facebook/presto/array/ReferenceCountMap.java
new file mode 100644
index 0000000000000..b0c2130442acb
--- /dev/null
+++ b/presto-array/src/main/java/com/facebook/presto/array/ReferenceCountMap.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.array;
+
+import io.airlift.slice.SizeOf;
+import it.unimi.dsi.fastutil.objects.Object2IntOpenCustomHashMap;
+import org.openjdk.jol.info.ClassLayout;
+
+public final class ReferenceCountMap
+ extends Object2IntOpenCustomHashMap
+{
+ private static final int INSTANCE_SIZE = ClassLayout.parseClass(ReferenceCountMap.class).instanceSize();
+
+ /**
+ * Two different blocks can share the same underlying data
+ * Use the map to avoid memory over counting
+ */
+ public ReferenceCountMap()
+ {
+ super(new ObjectStrategy());
+ }
+
+ /**
+ * Increments the reference count of an object by 1 and returns the updated reference count
+ */
+ public int incrementReference(Object key)
+ {
+ return addTo(key, 1) + 1;
+ }
+
+ /**
+ * Decrements the reference count of an object by 1 and returns the updated reference count
+ */
+ public int decrementReference(Object key)
+ {
+ int previousCount = addTo(key, -1);
+ if (previousCount == 1) {
+ remove(key);
+ }
+ return previousCount - 1;
+ }
+
+ /**
+ * Returns the size of this map in bytes.
+ */
+ public long sizeOf()
+ {
+ return INSTANCE_SIZE + SizeOf.sizeOf(key) + SizeOf.sizeOf(value) + SizeOf.sizeOf(used);
+ }
+
+ private static final class ObjectStrategy
+ implements Strategy
+ {
+ @Override
+ public int hashCode(Object object)
+ {
+ return System.identityHashCode(object);
+ }
+
+ @Override
+ public boolean equals(Object left, Object right)
+ {
+ return left == right;
+ }
+ }
+}
diff --git a/presto-array/src/main/java/com/facebook/presto/array/SliceBigArray.java b/presto-array/src/main/java/com/facebook/presto/array/SliceBigArray.java
index e757d6eb6a2df..6b620f4d67839 100644
--- a/presto-array/src/main/java/com/facebook/presto/array/SliceBigArray.java
+++ b/presto-array/src/main/java/com/facebook/presto/array/SliceBigArray.java
@@ -18,6 +18,7 @@
public final class SliceBigArray
{
+ private static final int INSTANCE_SIZE = ClassLayout.parseClass(SliceBigArray.class).instanceSize();
private static final int SLICE_INSTANCE_SIZE = ClassLayout.parseClass(Slice.class).instanceSize();
private final ObjectBigArray array;
private long sizeOfSlices;
@@ -37,7 +38,7 @@ public SliceBigArray(Slice slice)
*/
public long sizeOf()
{
- return array.sizeOf() + sizeOfSlices;
+ return INSTANCE_SIZE + array.sizeOf() + sizeOfSlices;
}
/**
diff --git a/presto-array/src/test/java/com/facebook/presto/array/TestBlockBigArray.java b/presto-array/src/test/java/com/facebook/presto/array/TestBlockBigArray.java
new file mode 100644
index 0000000000000..60d0a0f826c52
--- /dev/null
+++ b/presto-array/src/test/java/com/facebook/presto/array/TestBlockBigArray.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.array;
+
+import com.facebook.presto.spi.block.Block;
+import com.facebook.presto.spi.block.BlockBuilder;
+import com.facebook.presto.spi.block.BlockBuilderStatus;
+import com.facebook.presto.spi.block.IntArrayBlockBuilder;
+import org.openjdk.jol.info.ClassLayout;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+
+public class TestBlockBigArray
+{
+ @Test
+ public void testRetainedSizeWithOverlappingBlocks()
+ {
+ int entries = 123;
+ BlockBuilder blockBuilder = new IntArrayBlockBuilder(new BlockBuilderStatus(), entries);
+ for (int i = 0; i < entries; i++) {
+ blockBuilder.writeInt(i);
+ }
+ Block block = blockBuilder.build();
+
+ // Verify we do not over count
+ int arraySize = 456;
+ int blocks = 7890;
+ BlockBigArray blockBigArray = new BlockBigArray();
+ blockBigArray.ensureCapacity(arraySize);
+ for (int i = 0; i < blocks; i++) {
+ blockBigArray.set(i % arraySize, block.getRegion(0, entries));
+ }
+
+ ReferenceCountMap referenceCountMap = new ReferenceCountMap();
+ referenceCountMap.incrementReference(block);
+ long expectedSize = ClassLayout.parseClass(BlockBigArray.class).instanceSize()
+ + referenceCountMap.sizeOf()
+ + (new ObjectBigArray()).sizeOf()
+ + block.getRetainedSizeInBytes() + (arraySize - 1) * ClassLayout.parseClass(block.getClass()).instanceSize();
+ assertEquals(blockBigArray.sizeOf(), expectedSize);
+ }
+}
diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml
index b4afb5de4237c..c0384cd703970 100644
--- a/presto-atop/pom.xml
+++ b/presto-atop/pom.xml
@@ -5,7 +5,7 @@
com.facebook.presto
presto-root
- 0.175-SNAPSHOT
+ 0.180-SNAPSHOT
presto-atop
diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml
index 632d757d10a08..1e9275e84285e 100644
--- a/presto-base-jdbc/pom.xml
+++ b/presto-base-jdbc/pom.xml
@@ -5,7 +5,7 @@
com.facebook.presto
presto-root
- 0.175-SNAPSHOT
+ 0.180-SNAPSHOT
presto-base-jdbc
diff --git a/presto-base-jdbc/src/main/java/com/facebook/presto/plugin/jdbc/BaseJdbcConfig.java b/presto-base-jdbc/src/main/java/com/facebook/presto/plugin/jdbc/BaseJdbcConfig.java
index 49b08005ee9e6..5725721aab7ad 100644
--- a/presto-base-jdbc/src/main/java/com/facebook/presto/plugin/jdbc/BaseJdbcConfig.java
+++ b/presto-base-jdbc/src/main/java/com/facebook/presto/plugin/jdbc/BaseJdbcConfig.java
@@ -14,6 +14,7 @@
package com.facebook.presto.plugin.jdbc;
import io.airlift.configuration.Config;
+import io.airlift.configuration.ConfigSecuritySensitive;
import javax.validation.constraints.NotNull;
@@ -54,6 +55,7 @@ public String getConnectionPassword()
}
@Config("connection-password")
+ @ConfigSecuritySensitive
public BaseJdbcConfig setConnectionPassword(String connectionPassword)
{
this.connectionPassword = connectionPassword;
diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml
index 188bfcc22baf9..5da62b467f74e 100644
--- a/presto-benchmark-driver/pom.xml
+++ b/presto-benchmark-driver/pom.xml
@@ -5,7 +5,7 @@
com.facebook.presto
presto-root
- 0.175-SNAPSHOT
+ 0.180-SNAPSHOT
presto-benchmark-driver
@@ -72,6 +72,11 @@
commons-math3
+
+ com.squareup.okhttp3
+ okhttp
+
+
org.testng
diff --git a/presto-benchmark-driver/src/main/java/com/facebook/presto/benchmark/driver/BenchmarkDriverOptions.java b/presto-benchmark-driver/src/main/java/com/facebook/presto/benchmark/driver/BenchmarkDriverOptions.java
index 0685b71820cb6..7e97b21ade278 100644
--- a/presto-benchmark-driver/src/main/java/com/facebook/presto/benchmark/driver/BenchmarkDriverOptions.java
+++ b/presto-benchmark-driver/src/main/java/com/facebook/presto/benchmark/driver/BenchmarkDriverOptions.java
@@ -110,7 +110,7 @@ private static URI parseServer(String server)
HostAndPort host = HostAndPort.fromString(server);
try {
- return new URI("http", null, host.getHostText(), host.getPortOrDefault(80), null, null, null);
+ return new URI("http", null, host.getHost(), host.getPortOrDefault(80), null, null, null);
}
catch (URISyntaxException e) {
throw new IllegalArgumentException(e);
diff --git a/presto-benchmark-driver/src/main/java/com/facebook/presto/benchmark/driver/BenchmarkQueryRunner.java b/presto-benchmark-driver/src/main/java/com/facebook/presto/benchmark/driver/BenchmarkQueryRunner.java
index 2ca469875c643..1f82be9d18f22 100644
--- a/presto-benchmark-driver/src/main/java/com/facebook/presto/benchmark/driver/BenchmarkQueryRunner.java
+++ b/presto-benchmark-driver/src/main/java/com/facebook/presto/benchmark/driver/BenchmarkQueryRunner.java
@@ -15,7 +15,6 @@
import com.facebook.presto.client.ClientSession;
import com.facebook.presto.client.QueryError;
-import com.facebook.presto.client.QueryResults;
import com.facebook.presto.client.StatementClient;
import com.facebook.presto.client.StatementStats;
import com.google.common.base.Throwables;
@@ -28,8 +27,8 @@
import io.airlift.http.client.JsonResponseHandler;
import io.airlift.http.client.Request;
import io.airlift.http.client.jetty.JettyHttpClient;
-import io.airlift.json.JsonCodec;
import io.airlift.units.Duration;
+import okhttp3.OkHttpClient;
import java.io.Closeable;
import java.net.URI;
@@ -39,6 +38,7 @@
import static com.facebook.presto.benchmark.driver.BenchmarkQueryResult.failResult;
import static com.facebook.presto.benchmark.driver.BenchmarkQueryResult.passResult;
+import static com.facebook.presto.client.OkHttpUtil.setupSocksProxy;
import static com.google.common.base.Preconditions.checkArgument;
import static io.airlift.http.client.HttpUriBuilder.uriBuilderFrom;
import static io.airlift.http.client.JsonResponseHandler.createJsonResponseHandler;
@@ -59,8 +59,8 @@ public class BenchmarkQueryRunner
private final int maxFailures;
private final HttpClient httpClient;
+ private final OkHttpClient okHttpClient;
private final List nodes;
- private final JsonCodec queryResultsCodec;
private int failures;
@@ -77,8 +77,6 @@ public BenchmarkQueryRunner(int warm, int runs, boolean debug, int maxFailures,
this.debug = debug;
- this.queryResultsCodec = jsonCodec(QueryResults.class);
-
requireNonNull(socksProxy, "socksProxy is null");
HttpClientConfig httpClientConfig = new HttpClientConfig();
if (socksProxy.isPresent()) {
@@ -87,6 +85,10 @@ public BenchmarkQueryRunner(int warm, int runs, boolean debug, int maxFailures,
this.httpClient = new JettyHttpClient(httpClientConfig.setConnectTimeout(new Duration(10, TimeUnit.SECONDS)));
+ OkHttpClient.Builder builder = new OkHttpClient.Builder();
+ setupSocksProxy(builder, socksProxy);
+ this.okHttpClient = builder.build();
+
nodes = getAllNodes(requireNonNull(serverUri, "serverUri is null"));
}
@@ -149,7 +151,7 @@ public List getSchemas(ClientSession session)
failures = 0;
while (true) {
// start query
- StatementClient client = new StatementClient(httpClient, queryResultsCodec, session, "show schemas");
+ StatementClient client = new StatementClient(okHttpClient, session, "show schemas");
// read query output
ImmutableList.Builder schemas = ImmutableList.builder();
@@ -190,7 +192,7 @@ public List getSchemas(ClientSession session)
private StatementStats execute(ClientSession session, String name, String query)
{
// start query
- StatementClient client = new StatementClient(httpClient, queryResultsCodec, session, query);
+ StatementClient client = new StatementClient(okHttpClient, session, query);
// read query output
while (client.isValid() && client.advance()) {
diff --git a/presto-benchmark-driver/src/main/java/com/facebook/presto/benchmark/driver/Suite.java b/presto-benchmark-driver/src/main/java/com/facebook/presto/benchmark/driver/Suite.java
index 8d15446a3e19b..282236e91ab7b 100644
--- a/presto-benchmark-driver/src/main/java/com/facebook/presto/benchmark/driver/Suite.java
+++ b/presto-benchmark-driver/src/main/java/com/facebook/presto/benchmark/driver/Suite.java
@@ -26,11 +26,11 @@
import java.util.Map.Entry;
import java.util.Optional;
import java.util.regex.Pattern;
-import java.util.stream.Collectors;
-import java.util.stream.StreamSupport;
import static com.google.common.base.MoreObjects.toStringHelper;
import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.collect.ImmutableList.toImmutableList;
+import static com.google.common.collect.Streams.stream;
import static io.airlift.json.JsonCodec.mapJsonCodec;
import static java.util.Objects.requireNonNull;
@@ -93,11 +93,11 @@ public List selectQueries(Iterable queries)
return ImmutableList.copyOf(queries);
}
- List filteredQueries = StreamSupport.stream(queries.spliterator(), false)
+ List filteredQueries = stream(queries)
.filter(query -> getQueryNamePatterns().stream().anyMatch(pattern -> pattern.matcher(query.getName()).matches()))
- .collect(Collectors.toList());
+ .collect(toImmutableList());
- return ImmutableList.copyOf(filteredQueries);
+ return filteredQueries;
}
@Override
diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml
index 24d0a214a70a4..906fb82e54549 100644
--- a/presto-benchmark/pom.xml
+++ b/presto-benchmark/pom.xml
@@ -5,7 +5,7 @@
presto-root
com.facebook.presto
- 0.175-SNAPSHOT
+ 0.180-SNAPSHOT
presto-benchmark
diff --git a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/AbstractBenchmark.java b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/AbstractBenchmark.java
index f129fab0ac087..f931743a55bd8 100644
--- a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/AbstractBenchmark.java
+++ b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/AbstractBenchmark.java
@@ -123,10 +123,13 @@ public void runBenchmark(@Nullable BenchmarkResultHook benchmarkResultHook)
long outputRows = resultsAvg.get("output_rows").longValue();
DataSize outputBytes = new DataSize(resultsAvg.get("output_bytes"), BYTE);
- System.out.printf("%35s :: %8.3f cpu ms :: in %5s, %6s, %8s, %8s :: out %5s, %6s, %8s, %8s%n",
+ DataSize memory = new DataSize(resultsAvg.get("peak_memory"), BYTE);
+ System.out.printf("%35s :: %8.3f cpu ms :: %5s peak memory :: in %5s, %6s, %8s, %8s :: out %5s, %6s, %8s, %8s%n",
getBenchmarkName(),
cpuNanos.getValue(MILLISECONDS),
+ formatDataSize(memory, true),
+
formatCount(inputRows),
formatDataSize(inputBytes, true),
formatCountRate(inputRows, cpuNanos, true),
diff --git a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/AbstractOperatorBenchmark.java b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/AbstractOperatorBenchmark.java
index 9dd25dfb59b46..4d3fa50b2c6fa 100644
--- a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/AbstractOperatorBenchmark.java
+++ b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/AbstractOperatorBenchmark.java
@@ -101,21 +101,28 @@ protected OperatorFactory createHashProjectOperator(int operatorId, PlanNodeId p
protected abstract List createDrivers(TaskContext taskContext);
- protected void execute(TaskContext taskContext)
+ protected Map execute(TaskContext taskContext)
{
List drivers = createDrivers(taskContext);
+ long peakMemory = 0;
boolean done = false;
while (!done) {
boolean processed = false;
for (Driver driver : drivers) {
if (!driver.isFinished()) {
driver.process();
+ long lastPeakMemory = peakMemory;
+ peakMemory = (long) taskContext.getTaskStats().getMemoryReservation().getValue(BYTE);
+ if (peakMemory <= lastPeakMemory) {
+ peakMemory = lastPeakMemory;
+ }
processed = true;
}
}
done = !processed;
}
+ return ImmutableMap.of("peak_memory", peakMemory);
}
@Override
@@ -136,7 +143,7 @@ protected Map runOnce()
false);
CpuTimer cpuTimer = new CpuTimer();
- execute(taskContext);
+ Map executionStats = execute(taskContext);
CpuDuration executionTime = cpuTimer.elapsedTime();
TaskStats taskStats = taskContext.getTaskStats();
@@ -149,6 +156,7 @@ protected Map runOnce()
return ImmutableMap.builder()
// legacy computed values
+ .putAll(executionStats)
.put("elapsed_millis", executionTime.getWall().toMillis())
.put("input_rows_per_second", (long) (inputRows / executionTime.getWall().getValue(SECONDS)))
.put("output_rows_per_second", (long) (outputRows / executionTime.getWall().getValue(SECONDS)))
diff --git a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/ArrayAggregationBenchmark.java b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/ArrayAggregationBenchmark.java
new file mode 100644
index 0000000000000..13a2ad76e21ba
--- /dev/null
+++ b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/ArrayAggregationBenchmark.java
@@ -0,0 +1,32 @@
+/*
+ * 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.benchmark;
+
+import com.facebook.presto.testing.LocalQueryRunner;
+
+import static com.facebook.presto.benchmark.BenchmarkQueryRunner.createLocalQueryRunner;
+
+public class ArrayAggregationBenchmark
+ extends AbstractSqlBenchmark
+{
+ public ArrayAggregationBenchmark(LocalQueryRunner localQueryRunner)
+ {
+ super(localQueryRunner, "sql_double_array_agg", 10, 100, "select array_agg(totalprice) from orders group by orderkey");
+ }
+
+ public static void main(String[] args)
+ {
+ new ArrayAggregationBenchmark(createLocalQueryRunner()).runBenchmark(new SimpleLineBenchmarkResultWriter(System.out));
+ }
+}
diff --git a/presto-benchmark/src/test/java/com/facebook/presto/benchmark/BenchmarkInequalityJoin.java b/presto-benchmark/src/test/java/com/facebook/presto/benchmark/BenchmarkInequalityJoin.java
index e6136949498f8..4f46cbdba4293 100644
--- a/presto-benchmark/src/test/java/com/facebook/presto/benchmark/BenchmarkInequalityJoin.java
+++ b/presto-benchmark/src/test/java/com/facebook/presto/benchmark/BenchmarkInequalityJoin.java
@@ -59,7 +59,7 @@ public static class Context
private MemoryLocalQueryRunner queryRunner;
@Param({"true", "false"})
- private String fastInequalityJoin;
+ private String fastInequalityJoins;
// number of buckets. The smaller number of buckets, the longer position links chain
@Param({"100", "1000", "10000", "60000"})
@@ -78,7 +78,7 @@ public MemoryLocalQueryRunner getQueryRunner()
@Setup
public void setUp()
{
- queryRunner = new MemoryLocalQueryRunner(ImmutableMap.of(SystemSessionProperties.FAST_INEQUALITY_JOIN, fastInequalityJoin));
+ queryRunner = new MemoryLocalQueryRunner(ImmutableMap.of(SystemSessionProperties.FAST_INEQUALITY_JOINS, fastInequalityJoins));
// t1.val1 is in range [0, 1000)
// t1.bucket is in [0, 1000)
diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml
index 2af15a723f623..00a486eb29c58 100644
--- a/presto-benchto-benchmarks/pom.xml
+++ b/presto-benchto-benchmarks/pom.xml
@@ -4,7 +4,7 @@
com.facebook.presto
presto-root
- 0.175-SNAPSHOT
+ 0.180-SNAPSHOT
presto-benchto-benchmarks
diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml
index aead398ac4bbd..a931ab46f4f05 100644
--- a/presto-blackhole/pom.xml
+++ b/presto-blackhole/pom.xml
@@ -5,7 +5,7 @@
com.facebook.presto
presto-root
- 0.175-SNAPSHOT
+ 0.180-SNAPSHOT
presto-blackhole
diff --git a/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHolePageSourceProvider.java b/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHolePageSourceProvider.java
index 86bec9bbeb40c..abf1e2cdd75ae 100644
--- a/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHolePageSourceProvider.java
+++ b/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHolePageSourceProvider.java
@@ -145,7 +145,7 @@ else if (javaType == Slice.class) {
private boolean isSupportedType(Type type)
{
- return ImmutableSet.of(TINYINT, SMALLINT, INTEGER, BIGINT, REAL, DOUBLE, BOOLEAN, DATE, TIMESTAMP, VARBINARY).contains(type)
+ return ImmutableSet.of(TINYINT, SMALLINT, INTEGER, BIGINT, REAL, DOUBLE, BOOLEAN, DATE, TIMESTAMP, VARBINARY).contains(type)
|| isVarcharType(type) || isLongDecimal(type) || isShortDecimal(type);
}
}
diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml
index e0c248f5b0658..5630686676dac 100644
--- a/presto-bytecode/pom.xml
+++ b/presto-bytecode/pom.xml
@@ -5,7 +5,7 @@
com.facebook.presto
presto-root
- 0.175-SNAPSHOT
+ 0.180-SNAPSHOT
presto-bytecode
diff --git a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/ClassDefinition.java b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/ClassDefinition.java
index 3ea1903c52ff4..ad94f32a87ce4 100644
--- a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/ClassDefinition.java
+++ b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/ClassDefinition.java
@@ -25,6 +25,7 @@
import java.util.List;
import java.util.Set;
+import static com.facebook.presto.bytecode.Access.INTERFACE;
import static com.facebook.presto.bytecode.Access.STATIC;
import static com.facebook.presto.bytecode.Access.a;
import static com.facebook.presto.bytecode.Access.toAccessModifier;
@@ -64,7 +65,7 @@ public ClassDefinition(
ParameterizedType... interfaces)
{
requireNonNull(access, "access is null");
- requireNonNull(access, "access is null");
+ requireNonNull(type, "type is null");
requireNonNull(superClass, "superClass is null");
requireNonNull(interfaces, "interfaces is null");
@@ -121,6 +122,11 @@ public List getMethods()
return ImmutableList.copyOf(methods);
}
+ public boolean isInterface()
+ {
+ return access.contains(INTERFACE);
+ }
+
public void visit(ClassVisitor visitor)
{
// Generic signature if super class or any interface is generic
@@ -133,7 +139,8 @@ public void visit(ClassVisitor visitor)
for (int i = 0; i < interfaces.length; i++) {
interfaces[i] = this.interfaces.get(i).getClassName();
}
- visitor.visit(V1_7, toAccessModifier(access) | ACC_SUPER, type.getClassName(), signature, superClass.getClassName(), interfaces);
+ int accessModifier = toAccessModifier(access);
+ visitor.visit(V1_7, isInterface() ? accessModifier : accessModifier | ACC_SUPER, type.getClassName(), signature, superClass.getClassName(), interfaces);
// visit source
if (source != null) {
@@ -151,7 +158,9 @@ public void visit(ClassVisitor visitor)
}
// visit clinit method
- classInitializer.visit(visitor, true);
+ if (!isInterface()) {
+ classInitializer.visit(visitor, true);
+ }
// visit methods
for (MethodDefinition method : methods) {
@@ -210,6 +219,9 @@ public ClassDefinition addField(FieldDefinition field)
public MethodDefinition getClassInitializer()
{
+ if (isInterface()) {
+ throw new IllegalAccessError("Interface does not have class initializer");
+ }
return classInitializer;
}
diff --git a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/DumpBytecodeVisitor.java b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/DumpBytecodeVisitor.java
index 0062e073fa24b..9e0655275b382 100644
--- a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/DumpBytecodeVisitor.java
+++ b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/DumpBytecodeVisitor.java
@@ -50,6 +50,7 @@
import java.util.Collection;
import java.util.List;
+import static com.facebook.presto.bytecode.Access.INTERFACE;
import static com.facebook.presto.bytecode.ParameterizedType.type;
public class DumpBytecodeVisitor
@@ -72,7 +73,11 @@ public Void visitClass(ClassDefinition classDefinition)
}
// print class declaration
- Line classDeclaration = line().addAll(classDefinition.getAccess()).add("class").add(classDefinition.getType().getJavaClassName());
+ Line classDeclaration = line().addAll(classDefinition.getAccess());
+ if (!classDefinition.getAccess().contains(INTERFACE)) {
+ classDeclaration.add("class");
+ }
+ classDeclaration.add(classDefinition.getType().getJavaClassName());
if (!classDefinition.getSuperClass().equals(type(Object.class))) {
classDeclaration.add("extends").add(classDefinition.getSuperClass().getJavaClassName());
}
@@ -98,6 +103,9 @@ public Void visitClass(ClassDefinition classDefinition)
visitMethod(classDefinition, methodDefinition);
}
+ // print class initializer
+ visitMethod(classDefinition, classDefinition.getClassInitializer());
+
indentLevel--;
printLine("}");
printLine();
diff --git a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/MethodDefinition.java b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/MethodDefinition.java
index 5b651937091c5..0d0c50b015af8 100644
--- a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/MethodDefinition.java
+++ b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/MethodDefinition.java
@@ -84,9 +84,8 @@ public MethodDefinition(
this.parameters = ImmutableList.copyOf(parameters);
this.parameterTypes = Lists.transform(this.parameters, Parameter::getType);
this.parameterAnnotations = ImmutableList.copyOf(transform(parameters, input -> new ArrayList<>()));
-
Optional thisType = Optional.empty();
- if (!access.contains(STATIC)) {
+ if (!declaringClass.isInterface() && !access.contains(STATIC)) {
thisType = Optional.of(declaringClass.getType());
}
scope = new Scope(thisType, parameters);
@@ -171,6 +170,9 @@ public String getMethodDescriptor()
public BytecodeBlock getBody()
{
+ if (declaringClass.isInterface()) {
+ throw new IllegalAccessError("Interface does not have method body");
+ }
return body;
}
@@ -236,19 +238,19 @@ public void visit(ClassVisitor visitor, boolean addReturn)
parameterAnnotation.visitParameterAnnotation(parameterIndex, methodVisitor);
}
}
-
- // visit code
- methodVisitor.visitCode();
-
- // visit instructions
- MethodGenerationContext generationContext = new MethodGenerationContext(methodVisitor);
- generationContext.enterScope(scope);
- body.accept(methodVisitor, generationContext);
- if (addReturn) {
- new InsnNode(RETURN).accept(methodVisitor);
+ if (!declaringClass.isInterface()) {
+ // visit code
+ methodVisitor.visitCode();
+
+ // visit instructions
+ MethodGenerationContext generationContext = new MethodGenerationContext(methodVisitor);
+ generationContext.enterScope(scope);
+ body.accept(methodVisitor, generationContext);
+ if (addReturn) {
+ new InsnNode(RETURN).accept(methodVisitor);
+ }
+ generationContext.exitScope(scope);
}
- generationContext.exitScope(scope);
-
// done
methodVisitor.visitMaxs(-1, -1);
methodVisitor.visitEnd();
diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml
index bd8302c718cab..705b52946058f 100644
--- a/presto-cassandra/pom.xml
+++ b/presto-cassandra/pom.xml
@@ -4,7 +4,7 @@
com.facebook.presto
presto-root
- 0.175-SNAPSHOT
+ 0.180-SNAPSHOT
presto-cassandra
diff --git a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CachingCassandraSchemaProvider.java b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CachingCassandraSchemaProvider.java
deleted file mode 100644
index 92a637e8bb794..0000000000000
--- a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CachingCassandraSchemaProvider.java
+++ /dev/null
@@ -1,241 +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.cassandra;
-
-import com.facebook.presto.spi.NotFoundException;
-import com.facebook.presto.spi.SchemaNotFoundException;
-import com.facebook.presto.spi.SchemaTableName;
-import com.facebook.presto.spi.TableNotFoundException;
-import com.google.common.base.Throwables;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Maps;
-import com.google.common.util.concurrent.UncheckedExecutionException;
-import io.airlift.units.Duration;
-import org.weakref.jmx.Managed;
-
-import javax.annotation.concurrent.ThreadSafe;
-import javax.inject.Inject;
-
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-
-import static com.facebook.presto.cassandra.RetryDriver.retry;
-import static com.google.common.cache.CacheLoader.asyncReloading;
-import static java.util.Locale.ENGLISH;
-import static java.util.Objects.requireNonNull;
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
-
-/**
- * Cassandra Schema Cache
- */
-@ThreadSafe
-public class CachingCassandraSchemaProvider
-{
- private final String connectorId;
- private final CassandraSession session;
-
- /**
- * Mapping from an empty string to all schema names. Each schema name is a
- * mapping from the lower case schema name to the case sensitive schema name.
- * This mapping is necessary because Presto currently does not properly handle
- * case sensitive names.
- */
- private final LoadingCache> schemaNamesCache;
-
- /**
- * Mapping from lower case schema name to all tables in that schema. Each
- * table name is a mapping from the lower case table name to the case
- * sensitive table name. This mapping is necessary because Presto currently
- * does not properly handle case sensitive names.
- */
- private final LoadingCache> tableNamesCache;
- private final LoadingCache tableCache;
-
- @Inject
- public CachingCassandraSchemaProvider(
- CassandraConnectorId connectorId,
- CassandraSession session,
- @ForCassandra ExecutorService executor,
- CassandraClientConfig cassandraClientConfig)
- {
- this(requireNonNull(connectorId, "connectorId is null").toString(),
- session,
- executor,
- requireNonNull(cassandraClientConfig, "cassandraClientConfig is null").getSchemaCacheTtl(),
- cassandraClientConfig.getSchemaRefreshInterval());
- }
-
- public CachingCassandraSchemaProvider(String connectorId, CassandraSession session, ExecutorService executor, Duration cacheTtl, Duration refreshInterval)
- {
- this.connectorId = requireNonNull(connectorId, "connectorId is null");
- this.session = requireNonNull(session, "cassandraSession is null");
-
- requireNonNull(executor, "executor is null");
-
- long expiresAfterWriteMillis = requireNonNull(cacheTtl, "cacheTtl is null").toMillis();
- long refreshMills = requireNonNull(refreshInterval, "refreshInterval is null").toMillis();
-
- schemaNamesCache = CacheBuilder.newBuilder()
- .expireAfterWrite(expiresAfterWriteMillis, MILLISECONDS)
- .refreshAfterWrite(refreshMills, MILLISECONDS)
- .build(asyncReloading(new CacheLoader>()
- {
- @Override
- public Map load(String key)
- throws Exception
- {
- return loadAllSchemas();
- }
- }, executor));
-
- tableNamesCache = CacheBuilder.newBuilder()
- .expireAfterWrite(expiresAfterWriteMillis, MILLISECONDS)
- .refreshAfterWrite(refreshMills, MILLISECONDS)
- .build(asyncReloading(new CacheLoader>()
- {
- @Override
- public Map load(String databaseName)
- throws Exception
- {
- return loadAllTables(databaseName);
- }
- }, executor));
-
- tableCache = CacheBuilder.newBuilder()
- .expireAfterWrite(expiresAfterWriteMillis, MILLISECONDS)
- .refreshAfterWrite(refreshMills, MILLISECONDS)
- .build(asyncReloading(new CacheLoader()
- {
- @Override
- public CassandraTable load(SchemaTableName tableName)
- throws Exception
- {
- return loadTable(tableName);
- }
- }, executor));
- }
-
- @Managed
- public void flushCache()
- {
- schemaNamesCache.invalidateAll();
- tableNamesCache.invalidateAll();
- tableCache.invalidateAll();
- }
-
- public List getAllSchemas()
- {
- return ImmutableList.copyOf(getCacheValue(schemaNamesCache, "", RuntimeException.class).keySet());
- }
-
- private Map loadAllSchemas()
- throws Exception
- {
- return retry()
- .stopOnIllegalExceptions()
- .run("getAllSchemas", () -> Maps.uniqueIndex(session.getAllSchemas(), CachingCassandraSchemaProvider::toLowerCase));
- }
-
- public List getAllTables(String databaseName)
- throws SchemaNotFoundException
- {
- return ImmutableList.copyOf(getCacheValue(tableNamesCache, databaseName, SchemaNotFoundException.class).keySet());
- }
-
- private Map loadAllTables(final String databaseName)
- throws Exception
- {
- return retry().stopOn(NotFoundException.class).stopOnIllegalExceptions()
- .run("getAllTables", () -> {
- String caseSensitiveDatabaseName = getCaseSensitiveSchemaName(databaseName);
- if (caseSensitiveDatabaseName == null) {
- caseSensitiveDatabaseName = databaseName;
- }
- List tables = session.getAllTables(caseSensitiveDatabaseName);
- Map nameMap = Maps.uniqueIndex(tables, CachingCassandraSchemaProvider::toLowerCase);
-
- if (tables.isEmpty()) {
- // Check to see if the database exists
- session.getSchema(databaseName);
- }
- return nameMap;
- });
- }
-
- public CassandraTableHandle getTableHandle(SchemaTableName schemaTableName)
- {
- requireNonNull(schemaTableName, "schemaTableName is null");
- String schemaName = getCaseSensitiveSchemaName(schemaTableName.getSchemaName());
- String tableName = getCaseSensitiveTableName(schemaTableName);
- CassandraTableHandle tableHandle = new CassandraTableHandle(connectorId, schemaName, tableName);
- return tableHandle;
- }
-
- public String getCaseSensitiveSchemaName(String caseInsensitiveName)
- {
- String caseSensitiveSchemaName = getCacheValue(schemaNamesCache, "", RuntimeException.class).get(caseInsensitiveName.toLowerCase(ENGLISH));
- return caseSensitiveSchemaName == null ? caseInsensitiveName : caseSensitiveSchemaName;
- }
-
- public String getCaseSensitiveTableName(SchemaTableName schemaTableName)
- {
- String caseSensitiveTableName = getCacheValue(tableNamesCache, schemaTableName.getSchemaName(), SchemaNotFoundException.class).get(schemaTableName.getTableName().toLowerCase(ENGLISH));
- return caseSensitiveTableName == null ? schemaTableName.getTableName() : caseSensitiveTableName;
- }
-
- public CassandraTable getTable(CassandraTableHandle tableHandle)
- throws TableNotFoundException
- {
- return getCacheValue(tableCache, tableHandle.getSchemaTableName(), TableNotFoundException.class);
- }
-
- public void flushTable(SchemaTableName tableName)
- {
- tableCache.invalidate(tableName);
- tableNamesCache.invalidate(tableName.getSchemaName());
- schemaNamesCache.invalidateAll();
- }
-
- private CassandraTable loadTable(final SchemaTableName tableName)
- throws Exception
- {
- return retry()
- .stopOn(NotFoundException.class)
- .stopOnIllegalExceptions()
- .run("getTable", () -> session.getTable(tableName));
- }
-
- private static V getCacheValue(LoadingCache cache, K key, Class exceptionClass)
- throws E
- {
- try {
- return cache.get(key);
- }
- catch (ExecutionException | UncheckedExecutionException e) {
- Throwable t = e.getCause();
- Throwables.propagateIfInstanceOf(t, exceptionClass);
- throw Throwables.propagate(t);
- }
- }
-
- private static String toLowerCase(String value)
- {
- return value.toLowerCase(ENGLISH);
- }
-}
diff --git a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraClientConfig.java b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraClientConfig.java
index 40c5fd1f58b00..73a8a321df107 100644
--- a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraClientConfig.java
+++ b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraClientConfig.java
@@ -19,6 +19,7 @@
import com.google.common.collect.ImmutableList;
import io.airlift.configuration.Config;
import io.airlift.configuration.ConfigDescription;
+import io.airlift.configuration.ConfigSecuritySensitive;
import io.airlift.configuration.DefunctConfig;
import io.airlift.units.Duration;
import io.airlift.units.MaxDuration;
@@ -30,20 +31,17 @@
import java.util.Arrays;
import java.util.List;
-import java.util.concurrent.TimeUnit;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.MINUTES;
@DefunctConfig({"cassandra.thrift-port", "cassandra.partitioner", "cassandra.thrift-connection-factory-class", "cassandra.transport-factory-options",
- "cassandra.no-host-available-retry-count"})
+ "cassandra.no-host-available-retry-count", "cassandra.max-schema-refresh-threads", "cassandra.schema-cache-ttl",
+ "cassandra.schema-refresh-interval"})
public class CassandraClientConfig
{
private static final Splitter SPLITTER = Splitter.on(',').trimResults().omitEmptyStrings();
- private Duration schemaCacheTtl = new Duration(1, TimeUnit.HOURS);
- private Duration schemaRefreshInterval = new Duration(2, TimeUnit.MINUTES);
- private int maxSchemaRefreshThreads = 1;
private ConsistencyLevel consistencyLevel = ConsistencyLevel.ONE;
private int fetchSize = 5_000;
private List contactPoints = ImmutableList.of();
@@ -69,45 +67,6 @@ public class CassandraClientConfig
private int speculativeExecutionLimit = 1;
private Duration speculativeExecutionDelay = new Duration(500, MILLISECONDS);
- @Min(1)
- public int getMaxSchemaRefreshThreads()
- {
- return maxSchemaRefreshThreads;
- }
-
- @Config("cassandra.max-schema-refresh-threads")
- public CassandraClientConfig setMaxSchemaRefreshThreads(int maxSchemaRefreshThreads)
- {
- this.maxSchemaRefreshThreads = maxSchemaRefreshThreads;
- return this;
- }
-
- @NotNull
- public Duration getSchemaCacheTtl()
- {
- return schemaCacheTtl;
- }
-
- @Config("cassandra.schema-cache-ttl")
- public CassandraClientConfig setSchemaCacheTtl(Duration schemaCacheTtl)
- {
- this.schemaCacheTtl = schemaCacheTtl;
- return this;
- }
-
- @NotNull
- public Duration getSchemaRefreshInterval()
- {
- return schemaRefreshInterval;
- }
-
- @Config("cassandra.schema-refresh-interval")
- public CassandraClientConfig setSchemaRefreshInterval(Duration schemaRefreshInterval)
- {
- this.schemaRefreshInterval = schemaRefreshInterval;
- return this;
- }
-
@NotNull
@Size(min = 1)
public List getContactPoints()
@@ -224,6 +183,7 @@ public String getPassword()
}
@Config("cassandra.password")
+ @ConfigSecuritySensitive
public CassandraClientConfig setPassword(String password)
{
this.password = password;
diff --git a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraClientModule.java b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraClientModule.java
index d1cb67f7eee4c..141ab37d9d30d 100644
--- a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraClientModule.java
+++ b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraClientModule.java
@@ -35,17 +35,12 @@
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.List;
-import java.util.concurrent.ExecutorService;
import static com.google.common.base.Preconditions.checkArgument;
-import static io.airlift.concurrent.Threads.daemonThreadsNamed;
import static io.airlift.configuration.ConfigBinder.configBinder;
import static io.airlift.json.JsonCodecBinder.jsonCodecBinder;
import static java.lang.Math.toIntExact;
import static java.util.Objects.requireNonNull;
-import static java.util.concurrent.Executors.newFixedThreadPool;
-import static org.weakref.jmx.ObjectNames.generatedNameOf;
-import static org.weakref.jmx.guice.ExportBinder.newExporter;
public class CassandraClientModule
implements Module
@@ -71,22 +66,9 @@ public void configure(Binder binder)
configBinder(binder).bindConfig(CassandraClientConfig.class);
- binder.bind(CachingCassandraSchemaProvider.class).in(Scopes.SINGLETON);
- newExporter(binder).export(CachingCassandraSchemaProvider.class).as(generatedNameOf(CachingCassandraSchemaProvider.class, connectorId));
-
jsonCodecBinder(binder).bindListJsonCodec(ExtraColumnMetadata.class);
}
- @ForCassandra
- @Singleton
- @Provides
- public static ExecutorService createCachingCassandraSchemaExecutor(CassandraConnectorId clientId, CassandraClientConfig cassandraClientConfig)
- {
- return newFixedThreadPool(
- cassandraClientConfig.getMaxSchemaRefreshThreads(),
- daemonThreadsNamed("cassandra-" + clientId + "-%s"));
- }
-
@Singleton
@Provides
public static CassandraSession createCassandraSession(
@@ -161,6 +143,10 @@ public static CassandraSession createCassandraSession(
));
}
- return new NativeCassandraSession(connectorId.toString(), extraColumnMetadataCodec, clusterBuilder.build(), config.getNoHostAvailableRetryTimeout());
+ return new NativeCassandraSession(
+ connectorId.toString(),
+ extraColumnMetadataCodec,
+ new ReopeningCluster(clusterBuilder::build),
+ config.getNoHostAvailableRetryTimeout());
}
}
diff --git a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraClusteringPredicatesExtractor.java b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraClusteringPredicatesExtractor.java
index eecbbf6ae2f05..6693baff17b0f 100644
--- a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraClusteringPredicatesExtractor.java
+++ b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraClusteringPredicatesExtractor.java
@@ -13,66 +13,41 @@
*/
package com.facebook.presto.cassandra;
+import com.datastax.driver.core.VersionNumber;
import com.facebook.presto.cassandra.util.CassandraCqlUtils;
import com.facebook.presto.spi.ColumnHandle;
import com.facebook.presto.spi.predicate.Domain;
import com.facebook.presto.spi.predicate.Range;
import com.facebook.presto.spi.predicate.TupleDomain;
+import com.google.common.base.Joiner;
+import com.google.common.base.VerifyException;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
+import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
-import java.util.Set;
import static com.facebook.presto.cassandra.util.CassandraCqlUtils.toCQLCompatibleString;
-import static com.google.common.collect.Sets.cartesianProduct;
import static java.util.Objects.requireNonNull;
public class CassandraClusteringPredicatesExtractor
{
private final List clusteringColumns;
- private final TupleDomain predicates;
private final ClusteringPushDownResult clusteringPushDownResult;
+ private final TupleDomain predicates;
- public CassandraClusteringPredicatesExtractor(List clusteringColumns, TupleDomain predicates)
+ public CassandraClusteringPredicatesExtractor(List clusteringColumns, TupleDomain predicates, VersionNumber cassandraVersion)
{
- this.clusteringColumns = ImmutableList.copyOf(requireNonNull(clusteringColumns, "clusteringColumns is null"));
+ this.clusteringColumns = ImmutableList.copyOf(clusteringColumns);
this.predicates = requireNonNull(predicates, "predicates is null");
- this.clusteringPushDownResult = getClusteringKeysSet(clusteringColumns, predicates);
+ this.clusteringPushDownResult = getClusteringKeysSet(clusteringColumns, predicates, requireNonNull(cassandraVersion, "cassandraVersion is null"));
}
- public List getClusteringKeyPredicates()
+ public String getClusteringKeyPredicates()
{
- Set> pushedDownDomainValues = clusteringPushDownResult.getDomainValues();
-
- if (pushedDownDomainValues.isEmpty()) {
- return ImmutableList.of();
- }
-
- ImmutableList.Builder clusteringPredicates = ImmutableList.builder();
- for (List clusteringKeys : pushedDownDomainValues) {
- if (clusteringKeys.isEmpty()) {
- continue;
- }
-
- StringBuilder stringBuilder = new StringBuilder();
-
- for (int i = 0; i < clusteringKeys.size(); i++) {
- if (i > 0) {
- stringBuilder.append(" AND ");
- }
-
- stringBuilder.append(CassandraCqlUtils.validColumnName(clusteringColumns.get(i).getName()));
- stringBuilder.append(" = ");
- stringBuilder.append(CassandraCqlUtils.cqlValue(toCQLCompatibleString(clusteringKeys.get(i)), clusteringColumns.get(i).getCassandraType()));
- }
-
- clusteringPredicates.add(stringBuilder.toString());
- }
- return clusteringPredicates.build();
+ return clusteringPushDownResult.getDomainQuery();
}
public TupleDomain getUnenforcedConstraints()
@@ -87,65 +62,133 @@ public TupleDomain getUnenforcedConstraints()
return TupleDomain.withColumnDomains(notPushedDown);
}
- private static ClusteringPushDownResult getClusteringKeysSet(List clusteringColumns, TupleDomain predicates)
+ private static ClusteringPushDownResult getClusteringKeysSet(List clusteringColumns, TupleDomain predicates, VersionNumber cassandraVersion)
{
ImmutableMap.Builder domainsBuilder = ImmutableMap.builder();
- ImmutableList.Builder> clusteringColumnValues = ImmutableList.builder();
+ ImmutableList.Builder clusteringColumnSql = ImmutableList.builder();
+ int currentClusteringColumn = 0;
for (CassandraColumnHandle columnHandle : clusteringColumns) {
Domain domain = predicates.getDomains().get().get(columnHandle);
-
if (domain == null) {
break;
}
-
if (domain.isNullAllowed()) {
- return new ClusteringPushDownResult(domainsBuilder.build(), ImmutableSet.of());
+ break;
}
-
- Set values = domain.getValues().getValuesProcessor().transform(
+ String predicateString = null;
+ predicateString = domain.getValues().getValuesProcessor().transform(
ranges -> {
- ImmutableSet.Builder columnValues = ImmutableSet.builder();
- for (Range range : ranges.getOrderedRanges()) {
- if (!range.isSingleValue()) {
- return ImmutableSet.of();
+ List singleValues = new ArrayList<>();
+ List rangeConjuncts = new ArrayList<>();
+ String predicate = null;
+
+ for (Range range : ranges.getOrderedRanges()) {
+ if (range.isAll()) {
+ return null;
+ }
+ if (range.isSingleValue()) {
+ singleValues.add(CassandraCqlUtils.cqlValue(toCQLCompatibleString(range.getSingleValue()),
+ columnHandle.getCassandraType()));
+ }
+ else {
+ if (!range.getLow().isLowerUnbounded()) {
+ switch (range.getLow().getBound()) {
+ case ABOVE:
+ rangeConjuncts.add(CassandraCqlUtils.validColumnName(columnHandle.getName()) + " > "
+ + CassandraCqlUtils.cqlValue(toCQLCompatibleString(range.getLow().getValue()),
+ columnHandle.getCassandraType()));
+ break;
+ case EXACTLY:
+ rangeConjuncts.add(CassandraCqlUtils.validColumnName(columnHandle.getName()) + " >= "
+ + CassandraCqlUtils.cqlValue(toCQLCompatibleString(range.getLow().getValue()),
+ columnHandle.getCassandraType()));
+ break;
+ case BELOW:
+ throw new VerifyException("Low Marker should never use BELOW bound");
+ default:
+ throw new AssertionError("Unhandled bound: " + range.getLow().getBound());
}
- /* TODO add code to handle a range of values for the last column
- * Prior to Cassandra 2.2, only the last clustering column can have a range of values
- * Take a look at how this is done in PreparedStatementBuilder.java
- */
-
- Object value = range.getSingleValue();
-
- CassandraType valueType = columnHandle.getCassandraType();
- columnValues.add(valueType.validateClusteringKey(value));
}
- return columnValues.build();
- },
- discreteValues -> {
- if (discreteValues.isWhiteList()) {
- return ImmutableSet.copyOf(discreteValues.getValues());
+ if (!range.getHigh().isUpperUnbounded()) {
+ switch (range.getHigh().getBound()) {
+ case ABOVE:
+ throw new VerifyException("High Marker should never use ABOVE bound");
+ case EXACTLY:
+ rangeConjuncts.add(CassandraCqlUtils.validColumnName(columnHandle.getName()) + " <= "
+ + CassandraCqlUtils.cqlValue(toCQLCompatibleString(range.getHigh().getValue()),
+ columnHandle.getCassandraType()));
+ break;
+ case BELOW:
+ rangeConjuncts.add(CassandraCqlUtils.validColumnName(columnHandle.getName()) + " < "
+ + CassandraCqlUtils.cqlValue(toCQLCompatibleString(range.getHigh().getValue()),
+ columnHandle.getCassandraType()));
+ break;
+ default:
+ throw new AssertionError("Unhandled bound: " + range.getHigh().getBound());
+ }
}
- return ImmutableSet.of();
- },
- allOrNone -> ImmutableSet.of());
+ }
+ }
+
+ if (!singleValues.isEmpty() && !rangeConjuncts.isEmpty()) {
+ return null;
+ }
+ if (!singleValues.isEmpty()) {
+ if (singleValues.size() == 1) {
+ predicate = CassandraCqlUtils.validColumnName(columnHandle.getName()) + " = " + singleValues.get(0);
+ }
+ else {
+ predicate = CassandraCqlUtils.validColumnName(columnHandle.getName()) + " IN ("
+ + Joiner.on(",").join(singleValues) + ")";
+ }
+ }
+ else if (!rangeConjuncts.isEmpty()) {
+ predicate = Joiner.on(" AND ").join(rangeConjuncts);
+ }
+ return predicate;
+ }, discreteValues -> {
+ if (discreteValues.isWhiteList()) {
+ ImmutableList.Builder discreteValuesList = ImmutableList.builder();
+ for (Object discreteValue : discreteValues.getValues()) {
+ discreteValuesList.add(CassandraCqlUtils.cqlValue(toCQLCompatibleString(discreteValue),
+ columnHandle.getCassandraType()));
+ }
+ String predicate = CassandraCqlUtils.validColumnName(columnHandle.getName()) + " IN ("
+ + Joiner.on(",").join(discreteValuesList.build()) + ")";
+ return predicate;
+ }
+ return null;
+ }, allOrNone -> null);
- if (!values.isEmpty()) {
- clusteringColumnValues.add(values);
- domainsBuilder.put(columnHandle, domain);
+ if (predicateString == null) {
+ break;
+ }
+ // IN restriction only on last clustering column for Cassandra version = 2.1
+ if (predicateString.contains(" IN (") && cassandraVersion.compareTo(VersionNumber.parse("2.2.0")) < 0 && currentClusteringColumn != (clusteringColumns.size() - 1)) {
+ break;
+ }
+ clusteringColumnSql.add(predicateString);
+ domainsBuilder.put(columnHandle, domain);
+ // Check for last clustering column should only be restricted by range condition
+ if (predicateString.contains(">") || predicateString.contains("<")) {
+ break;
}
+ currentClusteringColumn++;
}
- return new ClusteringPushDownResult(domainsBuilder.build(), cartesianProduct(clusteringColumnValues.build()));
+ List clusteringColumnPredicates = clusteringColumnSql.build();
+
+ return new ClusteringPushDownResult(domainsBuilder.build(), Joiner.on(" AND ").join(clusteringColumnPredicates));
}
private static class ClusteringPushDownResult
{
private final Map domains;
- private final Set> domainValues;
+ private final String domainQuery;
- public ClusteringPushDownResult(Map domains, Set> domainValues)
+ public ClusteringPushDownResult(Map domains, String domainQuery)
{
this.domains = requireNonNull(ImmutableMap.copyOf(domains));
- this.domainValues = requireNonNull(ImmutableSet.copyOf(domainValues));
+ this.domainQuery = requireNonNull(domainQuery);
}
public Map getDomains()
@@ -153,9 +196,9 @@ public Map getDomains()
return domains;
}
- public Set> getDomainValues()
+ public String getDomainQuery()
{
- return domainValues;
+ return domainQuery;
}
}
}
diff --git a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraConnectorRecordSinkProvider.java b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraConnectorRecordSinkProvider.java
index 45eeded00c66a..fcd4dcd5fc86a 100644
--- a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraConnectorRecordSinkProvider.java
+++ b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraConnectorRecordSinkProvider.java
@@ -43,12 +43,28 @@ public RecordSink getRecordSink(ConnectorTransactionHandle transaction, Connecto
checkArgument(tableHandle instanceof CassandraOutputTableHandle, "tableHandle is not an instance of CassandraOutputTableHandle");
CassandraOutputTableHandle handle = (CassandraOutputTableHandle) tableHandle;
- return new CassandraRecordSink(handle, cassandraSession);
+ return new CassandraRecordSink(
+ cassandraSession,
+ handle.getSchemaName(),
+ handle.getTableName(),
+ handle.getColumnNames(),
+ handle.getColumnTypes(),
+ true);
}
@Override
public RecordSink getRecordSink(ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorInsertTableHandle tableHandle)
{
- throw new UnsupportedOperationException();
+ requireNonNull(tableHandle, "tableHandle is null");
+ checkArgument(tableHandle instanceof CassandraInsertTableHandle, "tableHandle is not an instance of ConnectorInsertTableHandle");
+ CassandraInsertTableHandle handle = (CassandraInsertTableHandle) tableHandle;
+
+ return new CassandraRecordSink(
+ cassandraSession,
+ handle.getSchemaName(),
+ handle.getTableName(),
+ handle.getColumnNames(),
+ handle.getColumnTypes(),
+ false);
}
}
diff --git a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraErrorCode.java b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraErrorCode.java
index 362739abca764..70d415ebebba8 100644
--- a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraErrorCode.java
+++ b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraErrorCode.java
@@ -22,7 +22,7 @@
public enum CassandraErrorCode
implements ErrorCodeSupplier
{
- CASSANDRA_METADATA_ERROR(0, EXTERNAL);
+ CASSANDRA_METADATA_ERROR(0, EXTERNAL), CASSANDRA_VERSION_ERROR(1, EXTERNAL);
private final ErrorCode errorCode;
diff --git a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraHandleResolver.java b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraHandleResolver.java
index f2fe188af7192..c82269ea0f22e 100644
--- a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraHandleResolver.java
+++ b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraHandleResolver.java
@@ -15,6 +15,7 @@
import com.facebook.presto.spi.ColumnHandle;
import com.facebook.presto.spi.ConnectorHandleResolver;
+import com.facebook.presto.spi.ConnectorInsertTableHandle;
import com.facebook.presto.spi.ConnectorOutputTableHandle;
import com.facebook.presto.spi.ConnectorSplit;
import com.facebook.presto.spi.ConnectorTableHandle;
@@ -59,4 +60,10 @@ public Class extends ConnectorTransactionHandle> getTransactionHandleClass()
{
return CassandraTransactionHandle.class;
}
+
+ @Override
+ public Class extends ConnectorInsertTableHandle> getInsertTableHandleClass()
+ {
+ return CassandraInsertTableHandle.class;
+ }
}
diff --git a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraInsertTableHandle.java b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraInsertTableHandle.java
new file mode 100644
index 0000000000000..57d1e55abeb04
--- /dev/null
+++ b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraInsertTableHandle.java
@@ -0,0 +1,90 @@
+/*
+ * 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.cassandra;
+
+import com.facebook.presto.spi.ConnectorInsertTableHandle;
+import com.facebook.presto.spi.type.Type;
+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 java.util.Objects.requireNonNull;
+
+public class CassandraInsertTableHandle
+ implements ConnectorInsertTableHandle
+{
+ private final String connectorId;
+ private final String schemaName;
+ private final String tableName;
+ private final List columnNames;
+ private final List columnTypes;
+
+ @JsonCreator
+ public CassandraInsertTableHandle(
+ @JsonProperty("connectorId") String connectorId,
+ @JsonProperty("schemaName") String schemaName,
+ @JsonProperty("tableName") String tableName,
+ @JsonProperty("columnNames") List columnNames,
+ @JsonProperty("columnTypes") List columnTypes)
+ {
+ this.connectorId = requireNonNull(connectorId, "clientId is null");
+ this.schemaName = requireNonNull(schemaName, "schemaName is null");
+ this.tableName = requireNonNull(tableName, "tableName is null");
+
+ requireNonNull(columnNames, "columnNames is null");
+ requireNonNull(columnTypes, "columnTypes is null");
+ checkArgument(columnNames.size() == columnTypes.size(), "columnNames and columnTypes sizes don't match");
+ this.columnNames = ImmutableList.copyOf(columnNames);
+ this.columnTypes = ImmutableList.copyOf(columnTypes);
+ }
+
+ @JsonProperty
+ public String getConnectorId()
+ {
+ return connectorId;
+ }
+
+ @JsonProperty
+ public String getSchemaName()
+ {
+ return schemaName;
+ }
+
+ @JsonProperty
+ public String getTableName()
+ {
+ return tableName;
+ }
+
+ @JsonProperty
+ public List getColumnNames()
+ {
+ return columnNames;
+ }
+
+ @JsonProperty
+ public List getColumnTypes()
+ {
+ return columnTypes;
+ }
+
+ @Override
+ public String toString()
+ {
+ return "cassandra:" + schemaName + "." + tableName;
+ }
+}
diff --git a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraMetadata.java b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraMetadata.java
index b3f465c84284e..3307f00f75bcd 100644
--- a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraMetadata.java
+++ b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraMetadata.java
@@ -16,6 +16,7 @@
import com.facebook.presto.cassandra.util.CassandraCqlUtils;
import com.facebook.presto.spi.ColumnHandle;
import com.facebook.presto.spi.ColumnMetadata;
+import com.facebook.presto.spi.ConnectorInsertTableHandle;
import com.facebook.presto.spi.ConnectorNewTableLayout;
import com.facebook.presto.spi.ConnectorOutputTableHandle;
import com.facebook.presto.spi.ConnectorSession;
@@ -30,6 +31,7 @@
import com.facebook.presto.spi.SchemaNotFoundException;
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.facebook.presto.spi.connector.ConnectorOutputMetadata;
import com.facebook.presto.spi.predicate.TupleDomain;
@@ -46,12 +48,16 @@
import java.util.Map;
import java.util.Optional;
import java.util.Set;
+import java.util.stream.Collectors;
import static com.facebook.presto.cassandra.CassandraType.toCassandraType;
+import static com.facebook.presto.cassandra.util.CassandraCqlUtils.validSchemaName;
+import static com.facebook.presto.cassandra.util.CassandraCqlUtils.validTableName;
import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED;
import static com.facebook.presto.spi.StandardErrorCode.PERMISSION_DENIED;
import static com.google.common.base.MoreObjects.toStringHelper;
import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.collect.ImmutableList.toImmutableList;
import static java.util.Locale.ENGLISH;
import static java.util.Objects.requireNonNull;
import static java.util.stream.Collectors.toList;
@@ -60,7 +66,6 @@ public class CassandraMetadata
implements ConnectorMetadata
{
private final String connectorId;
- private final CachingCassandraSchemaProvider schemaProvider;
private final CassandraSession cassandraSession;
private final CassandraPartitionManager partitionManager;
private final boolean allowDropTable;
@@ -68,15 +73,14 @@ public class CassandraMetadata
private final JsonCodec> extraColumnMetadataCodec;
@Inject
- public CassandraMetadata(CassandraConnectorId connectorId,
- CachingCassandraSchemaProvider schemaProvider,
+ public CassandraMetadata(
+ CassandraConnectorId connectorId,
CassandraSession cassandraSession,
CassandraPartitionManager partitionManager,
JsonCodec> extraColumnMetadataCodec,
CassandraClientConfig config)
{
this.connectorId = requireNonNull(connectorId, "connectorId is null").toString();
- this.schemaProvider = requireNonNull(schemaProvider, "schemaProvider is null");
this.partitionManager = requireNonNull(partitionManager, "partitionManager is null");
this.cassandraSession = requireNonNull(cassandraSession, "cassandraSession is null");
this.allowDropTable = requireNonNull(config, "config is null").getAllowDropTable();
@@ -86,7 +90,9 @@ public CassandraMetadata(CassandraConnectorId connectorId,
@Override
public List listSchemaNames(ConnectorSession session)
{
- return schemaProvider.getAllSchemas();
+ return cassandraSession.getCaseSensitiveSchemaNames().stream()
+ .map(name -> name.toLowerCase(ENGLISH))
+ .collect(toImmutableList());
}
@Override
@@ -94,11 +100,9 @@ public CassandraTableHandle getTableHandle(ConnectorSession session, SchemaTable
{
requireNonNull(tableName, "tableName is null");
try {
- CassandraTableHandle tableHandle = schemaProvider.getTableHandle(tableName);
- schemaProvider.getTable(tableHandle);
- return tableHandle;
+ return cassandraSession.getTable(tableName).getTableHandle();
}
- catch (NotFoundException e) {
+ catch (TableNotFoundException | SchemaNotFoundException e) {
// table was not found
return null;
}
@@ -113,15 +117,14 @@ private static SchemaTableName getTableName(ConnectorTableHandle tableHandle)
public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle tableHandle)
{
requireNonNull(tableHandle, "tableHandle is null");
- SchemaTableName tableName = getTableName(tableHandle);
- return getTableMetadata(session, tableName);
+ return getTableMetadata(getTableName(tableHandle));
}
- private ConnectorTableMetadata getTableMetadata(ConnectorSession session, SchemaTableName tableName)
+ private ConnectorTableMetadata getTableMetadata(SchemaTableName tableName)
{
- CassandraTableHandle tableHandle = schemaProvider.getTableHandle(tableName);
- List columns = getColumnHandles(session, tableHandle).values().stream()
- .map(column -> ((CassandraColumnHandle) column).getColumnMetadata())
+ CassandraTable table = cassandraSession.getTable(tableName);
+ List columns = table.getColumns().stream()
+ .map(CassandraColumnHandle::getColumnMetadata)
.collect(toList());
return new ConnectorTableMetadata(tableName, columns);
}
@@ -132,7 +135,7 @@ public List listTables(ConnectorSession session, String schemaN
ImmutableList.Builder tableNames = ImmutableList.builder();
for (String schemaName : listSchemas(session, schemaNameOrNull)) {
try {
- for (String tableName : schemaProvider.getAllTables(schemaName)) {
+ for (String tableName : cassandraSession.getCaseSensitiveTableNames(schemaName)) {
tableNames.add(new SchemaTableName(schemaName, tableName.toLowerCase(ENGLISH)));
}
}
@@ -154,7 +157,9 @@ private List listSchemas(ConnectorSession session, String schemaNameOrNu
@Override
public Map getColumnHandles(ConnectorSession session, ConnectorTableHandle tableHandle)
{
- CassandraTable table = schemaProvider.getTable((CassandraTableHandle) tableHandle);
+ requireNonNull(session, "session is null");
+ requireNonNull(tableHandle, "tableHandle is null");
+ CassandraTable table = cassandraSession.getTable(getTableName(tableHandle));
ImmutableMap.Builder columnHandles = ImmutableMap.builder();
for (CassandraColumnHandle columnHandle : table.getColumns()) {
columnHandles.put(CassandraCqlUtils.cqlNameToSqlName(columnHandle.getName()).toLowerCase(ENGLISH), columnHandle);
@@ -169,7 +174,7 @@ public Map> listTableColumns(ConnectorSess
ImmutableMap.Builder> columns = ImmutableMap.builder();
for (SchemaTableName tableName : listTables(session, prefix)) {
try {
- columns.put(tableName, getTableMetadata(session, tableName).getColumns());
+ columns.put(tableName, getTableMetadata(tableName).getColumns());
}
catch (NotFoundException e) {
// table disappeared during listing operation
@@ -198,16 +203,16 @@ public List getTableLayouts(ConnectorSession session
CassandraTableHandle handle = (CassandraTableHandle) table;
CassandraPartitionResult partitionResult = partitionManager.getPartitions(handle, constraint.getSummary());
- List clusteringKeyPredicates;
+ String clusteringKeyPredicates = "";
TupleDomain unenforcedConstraint;
if (partitionResult.isUnpartitioned()) {
- clusteringKeyPredicates = ImmutableList.of();
unenforcedConstraint = partitionResult.getUnenforcedConstraint();
}
else {
CassandraClusteringPredicatesExtractor clusteringPredicatesExtractor = new CassandraClusteringPredicatesExtractor(
- schemaProvider.getTable(handle).getClusteringKeyColumns(),
- partitionResult.getUnenforcedConstraint());
+ cassandraSession.getTable(getTableName(handle)).getClusteringKeyColumns(),
+ partitionResult.getUnenforcedConstraint(),
+ cassandraSession.getCassandraVersion());
clusteringKeyPredicates = clusteringPredicatesExtractor.getClusteringKeyPredicates();
unenforcedConstraint = clusteringPredicatesExtractor.getUnenforcedConstraints();
}
@@ -253,7 +258,6 @@ public void dropTable(ConnectorSession session, ConnectorTableHandle tableHandle
String tableName = cassandraTableHandle.getTableName();
cassandraSession.execute(String.format("DROP TABLE \"%s\".\"%s\"", schemaName, tableName));
- schemaProvider.flushTable(cassandraTableHandle.getSchemaTableName());
}
@Override
@@ -277,7 +281,7 @@ public ConnectorOutputTableHandle beginCreateTable(ConnectorSession session, Con
// get the root directory for the database
SchemaTableName table = tableMetadata.getTable();
- String schemaName = schemaProvider.getCaseSensitiveSchemaName(table.getSchemaName());
+ String schemaName = cassandraSession.getCaseSensitiveSchemaName(table.getSchemaName());
String tableName = table.getTableName();
List columns = columnNames.build();
List types = columnTypes.build();
@@ -309,8 +313,29 @@ public ConnectorOutputTableHandle beginCreateTable(ConnectorSession session, Con
@Override
public Optional finishCreateTable(ConnectorSession session, ConnectorOutputTableHandle tableHandle, Collection fragments)
{
- CassandraOutputTableHandle outputTableHandle = (CassandraOutputTableHandle) tableHandle;
- schemaProvider.flushTable(new SchemaTableName(outputTableHandle.getSchemaName(), outputTableHandle.getTableName()));
+ return Optional.empty();
+ }
+
+ @Override
+ public ConnectorInsertTableHandle beginInsert(ConnectorSession session, ConnectorTableHandle tableHandle)
+ {
+ CassandraTableHandle table = (CassandraTableHandle) tableHandle;
+ SchemaTableName schemaTableName = new SchemaTableName(table.getSchemaName(), table.getTableName());
+ List columns = cassandraSession.getTable(schemaTableName).getColumns();
+ List columnNames = columns.stream().map(CassandraColumnHandle::getName).map(CassandraCqlUtils::validColumnName).collect(Collectors.toList());
+ List columnTypes = columns.stream().map(CassandraColumnHandle::getType).collect(Collectors.toList());
+
+ return new CassandraInsertTableHandle(
+ connectorId,
+ validSchemaName(table.getSchemaName()),
+ validTableName(table.getTableName()),
+ columnNames,
+ columnTypes);
+ }
+
+ @Override
+ public Optional finishInsert(ConnectorSession session, ConnectorInsertTableHandle insertHandle, Collection fragments)
+ {
return Optional.empty();
}
}
diff --git a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraPartitionManager.java b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraPartitionManager.java
index de561dd581180..dec232d7b4e65 100644
--- a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraPartitionManager.java
+++ b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraPartitionManager.java
@@ -42,13 +42,11 @@ public class CassandraPartitionManager
{
private static final Logger log = Logger.get(CassandraPartitionManager.class);
- private final CachingCassandraSchemaProvider schemaProvider;
private final CassandraSession cassandraSession;
@Inject
- public CassandraPartitionManager(CachingCassandraSchemaProvider schemaProvider, CassandraSession cassandraSession)
+ public CassandraPartitionManager(CassandraSession cassandraSession)
{
- this.schemaProvider = requireNonNull(schemaProvider, "schemaProvider is null");
this.cassandraSession = requireNonNull(cassandraSession, "cassandraSession is null");
}
@@ -56,7 +54,7 @@ public CassandraPartitionResult getPartitions(ConnectorTableHandle tableHandle,
{
CassandraTableHandle cassandraTableHandle = (CassandraTableHandle) tableHandle;
- CassandraTable table = schemaProvider.getTable(cassandraTableHandle);
+ CassandraTable table = cassandraSession.getTable(cassandraTableHandle.getSchemaTableName());
List partitionKeys = table.getPartitionKeyColumns();
// fetch the partitions
diff --git a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraRecordSink.java b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraRecordSink.java
index 1ed416ead5f8a..52d26ba887c5c 100644
--- a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraRecordSink.java
+++ b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraRecordSink.java
@@ -13,6 +13,8 @@
*/
package com.facebook.presto.cassandra;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.querybuilder.Insert;
import com.facebook.presto.spi.RecordSink;
import com.facebook.presto.spi.type.Type;
import com.google.common.collect.ImmutableList;
@@ -20,17 +22,24 @@
import org.joda.time.format.DateTimeFormatter;
import org.joda.time.format.ISODateTimeFormat;
-import javax.inject.Inject;
-
+import java.nio.ByteBuffer;
+import java.sql.Timestamp;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
+import static com.datastax.driver.core.querybuilder.QueryBuilder.bindMarker;
+import static com.datastax.driver.core.querybuilder.QueryBuilder.insertInto;
+import static com.facebook.presto.spi.type.BigintType.BIGINT;
import static com.facebook.presto.spi.type.DateType.DATE;
import static com.facebook.presto.spi.type.IntegerType.INTEGER;
import static com.facebook.presto.spi.type.RealType.REAL;
+import static com.facebook.presto.spi.type.TimestampType.TIMESTAMP;
+import static com.facebook.presto.spi.type.VarbinaryType.VARBINARY;
+import static com.facebook.presto.spi.type.Varchars.isVarcharType;
+import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkState;
import static java.lang.Float.intBitsToFloat;
import static java.nio.charset.StandardCharsets.UTF_8;
@@ -41,37 +50,41 @@ public class CassandraRecordSink
{
private static final DateTimeFormatter DATE_FORMATTER = ISODateTimeFormat.date().withZoneUTC();
- private final int fieldCount;
private final CassandraSession cassandraSession;
- private final String insertQuery;
+ private final PreparedStatement insert;
private final List values;
private final List columnTypes;
+ private final boolean generateUUID;
+
private int field = -1;
- @Inject
- public CassandraRecordSink(CassandraOutputTableHandle handle, CassandraSession cassandraSession)
+ public CassandraRecordSink(
+ CassandraSession cassandraSession,
+ String schemaName,
+ String tableName,
+ List columnNames,
+ List columnTypes,
+ boolean generateUUID)
{
- this.fieldCount = requireNonNull(handle, "handle is null").getColumnNames().size();
- this.cassandraSession = requireNonNull(cassandraSession, "cassandraSession is null");
-
- String schemaName = handle.getSchemaName();
- StringBuilder queryBuilder = new StringBuilder(String.format("INSERT INTO \"%s\".\"%s\"(", schemaName, handle.getTableName()));
- queryBuilder.append("id");
-
- for (String columnName : handle.getColumnNames()) {
- queryBuilder.append(",").append(columnName);
+ this.cassandraSession = requireNonNull(cassandraSession, "cassandraSession");
+ requireNonNull(schemaName, "schemaName is null");
+ requireNonNull(tableName, "tableName is null");
+ requireNonNull(columnNames, "columnNames is null");
+ this.columnTypes = ImmutableList.copyOf(requireNonNull(columnTypes, "columnTypes is null"));
+ this.generateUUID = generateUUID;
+
+ Insert insert = insertInto(schemaName, tableName);
+ if (generateUUID) {
+ insert.value("id", bindMarker());
}
- queryBuilder.append(") VALUES (?");
-
- for (int i = 0; i < handle.getColumnNames().size(); i++) {
- queryBuilder.append(",?");
+ for (int i = 0; i < columnNames.size(); i++) {
+ String columnName = columnNames.get(i);
+ checkArgument(columnName != null, "columnName is null at position: %d", i);
+ insert.value(columnName, bindMarker());
}
- queryBuilder.append(")");
-
- insertQuery = queryBuilder.toString();
- values = new ArrayList<>();
+ this.insert = cassandraSession.prepare(insert);
- columnTypes = handle.getColumnTypes();
+ values = new ArrayList<>(columnTypes.size() + 1);
}
@Override
@@ -81,16 +94,18 @@ public void beginRecord()
field = 0;
values.clear();
- values.add(UUID.randomUUID());
+ if (generateUUID) {
+ values.add(UUID.randomUUID());
+ }
}
@Override
public void finishRecord()
{
checkState(field != -1, "not in record");
- checkState(field == fieldCount, "not all fields set");
+ checkState(field == columnTypes.size(), "not all fields set");
field = -1;
- cassandraSession.execute(insertQuery, values.toArray());
+ cassandraSession.execute(insert.bind(values.toArray()));
}
@Override
@@ -108,18 +123,25 @@ public void appendBoolean(boolean value)
@Override
public void appendLong(long value)
{
- if (DATE.equals(columnTypes.get(field))) {
+ Type columnType = columnTypes.get(field);
+ if (DATE.equals(columnType)) {
append(DATE_FORMATTER.print(TimeUnit.DAYS.toMillis(value)));
}
- else if (INTEGER.equals(columnTypes.get(field))) {
+ else if (INTEGER.equals(columnType)) {
append(((Number) value).intValue());
}
- else if (REAL.equals(columnTypes.get(field))) {
+ else if (REAL.equals(columnType)) {
append(intBitsToFloat((int) value));
}
- else {
+ else if (TIMESTAMP.equals(columnType)) {
+ append(new Timestamp(value));
+ }
+ else if (BIGINT.equals(columnType)) {
append(value);
}
+ else {
+ throw new UnsupportedOperationException("Type is not supported: " + columnType);
+ }
}
@Override
@@ -131,7 +153,16 @@ public void appendDouble(double value)
@Override
public void appendString(byte[] value)
{
- append(new String(value, UTF_8));
+ Type columnType = columnTypes.get(field);
+ if (VARBINARY.equals(columnType)) {
+ append(ByteBuffer.wrap(value));
+ }
+ else if (isVarcharType(columnType)) {
+ append(new String(value, UTF_8));
+ }
+ else {
+ throw new UnsupportedOperationException("Type is not supported: " + columnType);
+ }
}
@Override
@@ -160,7 +191,7 @@ public List getColumnTypes()
private void append(Object value)
{
checkState(field != -1, "not in record");
- checkState(field < fieldCount, "all fields already set");
+ checkState(field < columnTypes.size(), "all fields already set");
values.add(value);
field++;
}
diff --git a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraSession.java b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraSession.java
index 11eb426a54caa..fa5d3d5274b30 100644
--- a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraSession.java
+++ b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraSession.java
@@ -14,8 +14,12 @@
package com.facebook.presto.cassandra;
import com.datastax.driver.core.Host;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.RegularStatement;
import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Statement;
+import com.datastax.driver.core.TokenRange;
+import com.datastax.driver.core.VersionNumber;
import com.facebook.presto.spi.SchemaNotFoundException;
import com.facebook.presto.spi.SchemaTableName;
import com.facebook.presto.spi.TableNotFoundException;
@@ -28,30 +32,33 @@ public interface CassandraSession
{
String PRESTO_COMMENT_METADATA = "Presto Metadata:";
- Set getReplicas(String schemaName, ByteBuffer partitionKey);
+ VersionNumber getCassandraVersion();
- List getAllSchemas();
+ String getPartitioner();
- List getAllTables(String schema)
- throws SchemaNotFoundException;
+ Set getTokenRanges();
+
+ Set getReplicas(String caseSensitiveSchemaName, TokenRange tokenRange);
+
+ Set getReplicas(String caseSensitiveSchemaName, ByteBuffer partitionKey);
- void getSchema(String schema)
+ String getCaseSensitiveSchemaName(String caseInsensitiveSchemaName);
+
+ List getCaseSensitiveSchemaNames();
+
+ List getCaseSensitiveTableNames(String caseInsensitiveSchemaName)
throws SchemaNotFoundException;
- CassandraTable getTable(SchemaTableName tableName)
+ CassandraTable getTable(SchemaTableName schemaTableName)
throws TableNotFoundException;
List getPartitions(CassandraTable table, List filterPrefix);
- default ResultSet execute(String cql, Object... values)
- {
- return executeWithSession(session -> session.execute(cql, values));
- }
+ ResultSet execute(String cql, Object... values);
+
+ List getSizeEstimates(String keyspaceName, String tableName);
- T executeWithSession(SessionCallable sessionCallable);
+ PreparedStatement prepare(RegularStatement statement);
- interface SessionCallable
- {
- T executeWithSession(Session session);
- }
+ ResultSet execute(Statement statement);
}
diff --git a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraSplitManager.java b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraSplitManager.java
index e5d94b72c3c07..d0ce4dbbbb134 100644
--- a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraSplitManager.java
+++ b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraSplitManager.java
@@ -42,19 +42,17 @@ public class CassandraSplitManager
{
private final String connectorId;
private final CassandraSession cassandraSession;
- private final CachingCassandraSchemaProvider schemaProvider;
private final int partitionSizeForBatchSelect;
private final CassandraTokenSplitManager tokenSplitMgr;
@Inject
- public CassandraSplitManager(CassandraConnectorId connectorId,
+ public CassandraSplitManager(
+ CassandraConnectorId connectorId,
CassandraClientConfig cassandraClientConfig,
CassandraSession cassandraSession,
- CachingCassandraSchemaProvider schemaProvider,
CassandraTokenSplitManager tokenSplitMgr)
{
this.connectorId = requireNonNull(connectorId, "connectorId is null").toString();
- this.schemaProvider = requireNonNull(schemaProvider, "schemaProvider is null");
this.cassandraSession = requireNonNull(cassandraSession, "cassandraSession is null");
this.partitionSizeForBatchSelect = cassandraClientConfig.getPartitionSizeForBatchSelect();
this.tokenSplitMgr = tokenSplitMgr;
@@ -75,7 +73,7 @@ public ConnectorSplitSource getSplits(ConnectorTransactionHandle transaction, Co
if (partitions.size() == 1) {
CassandraPartition cassandraPartition = partitions.get(0);
if (cassandraPartition.isUnpartitioned() || cassandraPartition.isIndexedColumnPredicatePushdown()) {
- CassandraTable table = schemaProvider.getTable(cassandraTableHandle);
+ CassandraTable table = cassandraSession.getTable(cassandraTableHandle.getSchemaTableName());
List splits = getSplitsByTokenRange(table, cassandraPartition.getPartitionId());
return new FixedSplitSource(splits);
}
@@ -107,7 +105,7 @@ private static String buildTokenCondition(String tokenExpression, String startTo
return tokenExpression + " > " + startToken + " AND " + tokenExpression + " <= " + endToken;
}
- private List getSplitsForPartitions(CassandraTableHandle cassTableHandle, List partitions, List clusteringPredicates)
+ private List getSplitsForPartitions(CassandraTableHandle cassTableHandle, List partitions, String clusteringPredicates)
{
String schema = cassTableHandle.getSchemaName();
HostAddressFactory hostAddressFactory = new HostAddressFactory();
@@ -150,7 +148,7 @@ private List getSplitsForPartitions(CassandraTableHandle cassTab
hostMap.put(hostAddresses, addresses);
}
else {
- builder.addAll(createSplitsForClusteringPredicates(cassTableHandle, cassandraPartition.getPartitionId(), addresses, clusteringPredicates));
+ builder.add(createSplitForClusteringPredicates(cassTableHandle, cassandraPartition.getPartitionId(), addresses, clusteringPredicates));
}
}
if (singlePartitionKeyColumn) {
@@ -165,7 +163,7 @@ private List getSplitsForPartitions(CassandraTableHandle cassTab
size++;
if (size > partitionSizeForBatchSelect) {
String partitionId = String.format("%s in (%s)", partitionKeyColumnName, sb.toString());
- builder.addAll(createSplitsForClusteringPredicates(cassTableHandle, partitionId, hostMap.get(entry.getKey()), clusteringPredicates));
+ builder.add(createSplitForClusteringPredicates(cassTableHandle, partitionId, hostMap.get(entry.getKey()), clusteringPredicates));
size = 0;
sb.setLength(0);
sb.trimToSize();
@@ -173,31 +171,27 @@ private List getSplitsForPartitions(CassandraTableHandle cassTab
}
if (size > 0) {
String partitionId = String.format("%s in (%s)", partitionKeyColumnName, sb.toString());
- builder.addAll(createSplitsForClusteringPredicates(cassTableHandle, partitionId, hostMap.get(entry.getKey()), clusteringPredicates));
+ builder.add(createSplitForClusteringPredicates(cassTableHandle, partitionId, hostMap.get(entry.getKey()), clusteringPredicates));
}
}
}
return builder.build();
}
- private List createSplitsForClusteringPredicates(
+ private CassandraSplit createSplitForClusteringPredicates(
CassandraTableHandle tableHandle,
String partitionId,
List hosts,
- List clusteringPredicates)
+ String clusteringPredicates)
{
String schema = tableHandle.getSchemaName();
String table = tableHandle.getTableName();
if (clusteringPredicates.isEmpty()) {
- return ImmutableList.of(new CassandraSplit(connectorId, schema, table, partitionId, null, hosts));
+ return new CassandraSplit(connectorId, schema, table, partitionId, null, hosts);
}
- ImmutableList.Builder builder = ImmutableList.builder();
- for (String clusteringPredicate : clusteringPredicates) {
- builder.add(new CassandraSplit(connectorId, schema, table, partitionId, clusteringPredicate, hosts));
- }
- return builder.build();
+ return new CassandraSplit(connectorId, schema, table, partitionId, clusteringPredicates, hosts);
}
@Override
diff --git a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraTableLayoutHandle.java b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraTableLayoutHandle.java
index dfafda61ef6dc..fe2da9a38c013 100644
--- a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraTableLayoutHandle.java
+++ b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraTableLayoutHandle.java
@@ -28,19 +28,19 @@ public final class CassandraTableLayoutHandle
{
private final CassandraTableHandle table;
private final List partitions;
- private final List clusteringPredicates;
+ private final String clusteringPredicates;
@JsonCreator
public CassandraTableLayoutHandle(@JsonProperty("table") CassandraTableHandle table)
{
- this(table, ImmutableList.of(), ImmutableList.of());
+ this(table, ImmutableList.of(), "");
}
- public CassandraTableLayoutHandle(CassandraTableHandle table, List partitions, List clusteringPredicates)
+ public CassandraTableLayoutHandle(CassandraTableHandle table, List partitions, String clusteringPredicates)
{
this.table = requireNonNull(table, "table is null");
this.partitions = ImmutableList.copyOf(requireNonNull(partitions, "partition is null"));
- this.clusteringPredicates = ImmutableList.copyOf(requireNonNull(clusteringPredicates, "clusteringPredicates is null"));
+ this.clusteringPredicates = requireNonNull(clusteringPredicates, "clusteringPredicates is null");
}
@JsonProperty
@@ -56,7 +56,7 @@ public List getPartitions()
}
@JsonIgnore
- public List getClusteringPredicates()
+ public String getClusteringPredicates()
{
return clusteringPredicates;
}
diff --git a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraTokenSplitManager.java b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraTokenSplitManager.java
index 9c55f7fcfee87..cc9fe2e400fb9 100644
--- a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraTokenSplitManager.java
+++ b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraTokenSplitManager.java
@@ -14,11 +14,6 @@
package com.facebook.presto.cassandra;
import com.datastax.driver.core.Host;
-import com.datastax.driver.core.KeyspaceMetadata;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.Row;
-import com.datastax.driver.core.Statement;
-import com.datastax.driver.core.TableMetadata;
import com.datastax.driver.core.TokenRange;
import com.facebook.presto.spi.PrestoException;
import com.google.common.collect.ImmutableList;
@@ -32,11 +27,8 @@
import java.util.Set;
import java.util.concurrent.ThreadLocalRandom;
-import static com.datastax.driver.core.querybuilder.QueryBuilder.eq;
-import static com.datastax.driver.core.querybuilder.QueryBuilder.select;
import static com.facebook.presto.cassandra.CassandraErrorCode.CASSANDRA_METADATA_ERROR;
import static com.facebook.presto.cassandra.TokenRing.createForPartitioner;
-import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkState;
import static java.lang.Math.max;
@@ -49,9 +41,6 @@
public class CassandraTokenSplitManager
{
- private static final String SYSTEM = "system";
- private static final String SIZE_ESTIMATES = "size_estimates";
-
private final CassandraSession session;
private final int splitSize;
@@ -69,7 +58,7 @@ public CassandraTokenSplitManager(CassandraSession session, int splitSize)
public List getSplits(String keyspace, String table)
{
- Set tokenRanges = getTokenRanges();
+ Set tokenRanges = session.getTokenRanges();
if (tokenRanges.isEmpty()) {
throw new PrestoException(CASSANDRA_METADATA_ERROR, "The cluster metadata is not available. " +
@@ -81,7 +70,7 @@ public List getSplits(String keyspace, String table)
tokenRanges = unwrap(tokenRanges);
}
- Optional tokenRing = createForPartitioner(getPartitioner());
+ Optional tokenRing = createForPartitioner(session.getPartitioner());
long totalPartitionsCount = getTotalPartitionsCount(keyspace, table);
List splits = new ArrayList<>();
@@ -118,11 +107,6 @@ public List getSplits(String keyspace, String table)
return unmodifiableList(splits);
}
- private Set getTokenRanges()
- {
- return session.executeWithSession(session -> session.getCluster().getMetadata().getTokenRanges());
- }
-
private Set unwrap(Set tokenRanges)
{
ImmutableSet.Builder result = ImmutableSet.builder();
@@ -134,58 +118,20 @@ private Set unwrap(Set tokenRanges)
private long getTotalPartitionsCount(String keyspace, String table)
{
- List estimates = getSizeEstimates(keyspace, table);
+ List estimates = session.getSizeEstimates(keyspace, table);
return estimates.stream()
.mapToLong(SizeEstimate::getPartitionsCount)
.sum();
}
- private List getSizeEstimates(String keyspaceName, String tableName)
- {
- checkSizeEstimatesTableExist();
-
- Statement statement = select("range_start", "range_end", "mean_partition_size", "partitions_count")
- .from(SYSTEM, SIZE_ESTIMATES)
- .where(eq("keyspace_name", keyspaceName))
- .and(eq("table_name", tableName));
-
- ResultSet result = session.executeWithSession(session -> session.execute(statement));
- ImmutableList.Builder estimates = ImmutableList.builder();
- for (Row row : result.all()) {
- SizeEstimate estimate = new SizeEstimate(
- row.getString("range_start"),
- row.getString("range_end"),
- row.getLong("mean_partition_size"),
- row.getLong("partitions_count"));
- estimates.add(estimate);
- }
-
- return estimates.build();
- }
-
- private void checkSizeEstimatesTableExist()
- {
- KeyspaceMetadata ks = session.executeWithSession(session -> session.getCluster().getMetadata().getKeyspace(SYSTEM));
- checkState(ks != null, "system keyspace metadata must not be null");
- TableMetadata table = ks.getTable(SIZE_ESTIMATES);
- if (table == null) {
- throw new PrestoException(NOT_SUPPORTED, "Cassandra versions prior to 2.1.5 are not supported");
- }
- }
-
private List getEndpoints(String keyspace, TokenRange tokenRange)
{
- Set endpoints = session.executeWithSession(session -> session.getCluster().getMetadata().getReplicas(keyspace, tokenRange));
+ Set endpoints = session.getReplicas(keyspace, tokenRange);
return unmodifiableList(endpoints.stream()
.map(Host::toString)
.collect(toList()));
}
- private String getPartitioner()
- {
- return session.executeWithSession(session -> session.getCluster().getMetadata().getPartitioner());
- }
-
private static TokenSplit createSplit(TokenRange range, List endpoints)
{
checkArgument(!range.isEmpty(), "tokenRange must not be empty");
@@ -222,40 +168,4 @@ public List getHosts()
return hosts;
}
}
-
- private static class SizeEstimate
- {
- private final String rangeStart;
- private final String rangeEnd;
- private final long meanPartitionSize;
- private final long partitionsCount;
-
- public SizeEstimate(String rangeStart, String rangeEnd, long meanPartitionSize, long partitionsCount)
- {
- this.rangeStart = requireNonNull(rangeStart, "rangeStart is null");
- this.rangeEnd = requireNonNull(rangeEnd, "rangeEnd is null");
- this.meanPartitionSize = meanPartitionSize;
- this.partitionsCount = partitionsCount;
- }
-
- public String getRangeStart()
- {
- return rangeStart;
- }
-
- public String getRangeEnd()
- {
- return rangeEnd;
- }
-
- public long getMeanPartitionSize()
- {
- return meanPartitionSize;
- }
-
- public long getPartitionsCount()
- {
- return partitionsCount;
- }
- }
}
diff --git a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/NativeCassandraSession.java b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/NativeCassandraSession.java
index d46b0a79f2135..bd08ff21fdbfb 100644
--- a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/NativeCassandraSession.java
+++ b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/NativeCassandraSession.java
@@ -19,9 +19,15 @@
import com.datastax.driver.core.Host;
import com.datastax.driver.core.IndexMetadata;
import com.datastax.driver.core.KeyspaceMetadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.RegularStatement;
+import com.datastax.driver.core.ResultSet;
import com.datastax.driver.core.Row;
import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Statement;
import com.datastax.driver.core.TableMetadata;
+import com.datastax.driver.core.TokenRange;
+import com.datastax.driver.core.VersionNumber;
import com.datastax.driver.core.exceptions.NoHostAvailableException;
import com.datastax.driver.core.policies.ReconnectionPolicy;
import com.datastax.driver.core.policies.ReconnectionPolicy.ReconnectionSchedule;
@@ -30,6 +36,7 @@
import com.datastax.driver.core.querybuilder.Select;
import com.facebook.presto.cassandra.util.CassandraCqlUtils;
import com.facebook.presto.spi.ColumnHandle;
+import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.SchemaNotFoundException;
import com.facebook.presto.spi.SchemaTableName;
import com.facebook.presto.spi.TableNotFoundException;
@@ -46,29 +53,41 @@
import java.nio.ByteBuffer;
import java.util.ArrayList;
+import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
+import java.util.Map;
import java.util.Set;
+import static com.datastax.driver.core.querybuilder.QueryBuilder.eq;
+import static com.datastax.driver.core.querybuilder.QueryBuilder.select;
import static com.datastax.driver.core.querybuilder.Select.Where;
+import static com.facebook.presto.cassandra.CassandraErrorCode.CASSANDRA_VERSION_ERROR;
+import static com.facebook.presto.cassandra.util.CassandraCqlUtils.validSchemaName;
+import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED;
+import static com.google.common.base.Preconditions.checkState;
import static com.google.common.base.Predicates.in;
import static com.google.common.base.Predicates.not;
import static com.google.common.base.Suppliers.memoize;
import static com.google.common.collect.Iterables.filter;
import static com.google.common.collect.Iterables.transform;
+import static java.lang.String.format;
import static java.util.Comparator.comparing;
+import static java.util.Locale.ENGLISH;
import static java.util.Objects.requireNonNull;
import static java.util.stream.Collectors.toList;
-// TODO: Refactor this class to make it be "single responsibility"
public class NativeCassandraSession
implements CassandraSession
{
private static final Logger log = Logger.get(NativeCassandraSession.class);
- static final String PRESTO_COMMENT_METADATA = "Presto Metadata:";
- protected final String connectorId;
+ private static final String PRESTO_COMMENT_METADATA = "Presto Metadata:";
+ private static final String SYSTEM = "system";
+ private static final String SIZE_ESTIMATES = "size_estimates";
+
+ private final String connectorId;
private final JsonCodec> extraColumnMetadataCodec;
private final Cluster cluster;
private final Supplier session;
@@ -84,13 +103,56 @@ public NativeCassandraSession(String connectorId, JsonCodec getReplicas(String schemaName, ByteBuffer partitionKey)
+ public VersionNumber getCassandraVersion()
+ {
+ ResultSet result = executeWithSession(session -> session.execute("select release_version from system.local"));
+ Row versionRow = result.one();
+ if (versionRow == null) {
+ throw new PrestoException(CASSANDRA_VERSION_ERROR, "The cluster version is not available. " +
+ "Please make sure that the Cassandra cluster is up and running, " +
+ "and that the contact points are specified correctly.");
+ }
+ return VersionNumber.parse(versionRow.getString("release_version"));
+ }
+
+ @Override
+ public String getPartitioner()
+ {
+ return executeWithSession(session -> session.getCluster().getMetadata().getPartitioner());
+ }
+
+ @Override
+ public Set getTokenRanges()
+ {
+ return executeWithSession(session -> session.getCluster().getMetadata().getTokenRanges());
+ }
+
+ @Override
+ public Set getReplicas(String caseSensitiveSchemaName, TokenRange tokenRange)
+ {
+ requireNonNull(caseSensitiveSchemaName, "keyspace is null");
+ requireNonNull(tokenRange, "tokenRange is null");
+ return executeWithSession(session ->
+ session.getCluster().getMetadata().getReplicas(validSchemaName(caseSensitiveSchemaName), tokenRange));
+ }
+
+ @Override
+ public Set getReplicas(String caseSensitiveSchemaName, ByteBuffer partitionKey)
+ {
+ requireNonNull(caseSensitiveSchemaName, "keyspace is null");
+ requireNonNull(partitionKey, "partitionKey is null");
+ return executeWithSession(session ->
+ session.getCluster().getMetadata().getReplicas(validSchemaName(caseSensitiveSchemaName), partitionKey));
+ }
+
+ @Override
+ public String getCaseSensitiveSchemaName(String caseInsensitiveSchemaName)
{
- return executeWithSession(session -> session.getCluster().getMetadata().getReplicas(schemaName, partitionKey));
+ return getKeyspaceByCaseInsensitiveName(caseInsensitiveSchemaName).getName();
}
@Override
- public List getAllSchemas()
+ public List getCaseSensitiveSchemaNames()
{
ImmutableList.Builder builder = ImmutableList.builder();
List keyspaces = executeWithSession(session -> session.getCluster().getMetadata().getKeyspaces());
@@ -101,42 +163,28 @@ public List getAllSchemas()
}
@Override
- public List getAllTables(String schema)
+ public List getCaseSensitiveTableNames(String caseInsensitiveSchemaName)
throws SchemaNotFoundException
{
- KeyspaceMetadata meta = getCheckedKeyspaceMetadata(schema);
+ KeyspaceMetadata keyspace = getKeyspaceByCaseInsensitiveName(caseInsensitiveSchemaName);
ImmutableList.Builder builder = ImmutableList.builder();
- for (TableMetadata tableMeta : meta.getTables()) {
- builder.add(tableMeta.getName());
+ for (TableMetadata table : keyspace.getTables()) {
+ builder.add(table.getName());
}
return builder.build();
}
- private KeyspaceMetadata getCheckedKeyspaceMetadata(String schema)
- throws SchemaNotFoundException
- {
- KeyspaceMetadata keyspaceMetadata = executeWithSession(session -> session.getCluster().getMetadata().getKeyspace(schema));
- if (keyspaceMetadata == null) {
- throw new SchemaNotFoundException(schema);
- }
- return keyspaceMetadata;
- }
-
- @Override
- public void getSchema(String schema)
- throws SchemaNotFoundException
- {
- getCheckedKeyspaceMetadata(schema);
- }
-
@Override
- public CassandraTable getTable(SchemaTableName tableName)
+ public CassandraTable getTable(SchemaTableName schemaTableName)
throws TableNotFoundException
{
- TableMetadata tableMeta = getTableMetadata(tableName);
+ KeyspaceMetadata keyspace = getKeyspaceByCaseInsensitiveName(schemaTableName.getSchemaName());
+ TableMetadata tableMeta = getTableMetadata(keyspace, schemaTableName.getTableName());
List columnNames = new ArrayList<>();
- for (ColumnMetadata columnMetadata : tableMeta.getColumns()) {
+ List columns = tableMeta.getColumns();
+ checkColumnNames(columns);
+ for (ColumnMetadata columnMetadata : columns) {
columnNames.add(columnMetadata.getName());
}
@@ -178,7 +226,7 @@ public CassandraTable getTable(SchemaTableName tableName)
}
// add other columns
- for (ColumnMetadata columnMeta : tableMeta.getColumns()) {
+ for (ColumnMetadata columnMeta : columns) {
if (!primaryKeySet.contains(columnMeta.getName())) {
boolean hidden = hiddenColumns.contains(columnMeta.getName());
CassandraColumnHandle columnHandle = buildColumnHandle(tableMeta, columnMeta, false, false, columnNames.indexOf(columnMeta.getName()), hidden);
@@ -194,23 +242,66 @@ public CassandraTable getTable(SchemaTableName tableName)
return new CassandraTable(tableHandle, sortedColumnHandles);
}
- private TableMetadata getTableMetadata(SchemaTableName schemaTableName)
+ private KeyspaceMetadata getKeyspaceByCaseInsensitiveName(String caseInsensitiveSchemaName)
+ throws SchemaNotFoundException
{
- String schemaName = schemaTableName.getSchemaName();
- String tableName = schemaTableName.getTableName();
+ List keyspaces = executeWithSession(session -> session.getCluster().getMetadata().getKeyspaces());
+ KeyspaceMetadata result = null;
+ // Ensure that the error message is deterministic
+ List sortedKeyspaces = Ordering.from(comparing(KeyspaceMetadata::getName)).immutableSortedCopy(keyspaces);
+ for (KeyspaceMetadata keyspace : sortedKeyspaces) {
+ if (keyspace.getName().equalsIgnoreCase(caseInsensitiveSchemaName)) {
+ if (result != null) {
+ throw new PrestoException(
+ NOT_SUPPORTED,
+ format("More than one keyspace has been found for the case insensitive schema name: %s -> (%s, %s)",
+ caseInsensitiveSchemaName, result.getName(), keyspace.getName()));
+ }
+ result = keyspace;
+ }
+ }
+ if (result == null) {
+ throw new SchemaNotFoundException(caseInsensitiveSchemaName);
+ }
+ return result;
+ }
- KeyspaceMetadata keyspaceMetadata = getCheckedKeyspaceMetadata(schemaName);
- TableMetadata tableMetadata = keyspaceMetadata.getTable(tableName);
- if (tableMetadata != null) {
- return tableMetadata;
+ private static TableMetadata getTableMetadata(KeyspaceMetadata keyspace, String caseInsensitiveTableName)
+ {
+ TableMetadata result = null;
+ Collection tables = keyspace.getTables();
+ // Ensure that the error message is deterministic
+ List sortedTables = Ordering.from(comparing(TableMetadata::getName)).immutableSortedCopy(tables);
+ for (TableMetadata table : sortedTables) {
+ if (table.getName().equalsIgnoreCase(caseInsensitiveTableName)) {
+ if (result != null) {
+ throw new PrestoException(
+ NOT_SUPPORTED,
+ format("More than one table has been found for the case insensitive table name: %s -> (%s, %s)",
+ caseInsensitiveTableName, result.getName(), table.getName()));
+ }
+ result = table;
+ }
}
+ if (result == null) {
+ throw new TableNotFoundException(new SchemaTableName(keyspace.getName(), caseInsensitiveTableName));
+ }
+ return result;
+ }
- for (TableMetadata table : keyspaceMetadata.getTables()) {
- if (table.getName().equalsIgnoreCase(tableName)) {
- return table;
+ private static void checkColumnNames(List columns)
+ {
+ Map lowercaseNameToColumnMap = new HashMap<>();
+ for (ColumnMetadata column : columns) {
+ String lowercaseName = column.getName().toLowerCase(ENGLISH);
+ if (lowercaseNameToColumnMap.containsKey(lowercaseName)) {
+ throw new PrestoException(
+ NOT_SUPPORTED,
+ format("More than one column has been found for the case insensitive column name: %s -> (%s, %s)",
+ lowercaseName, lowercaseNameToColumnMap.get(lowercaseName).getName(), column.getName()));
}
+ lowercaseNameToColumnMap.put(lowercaseName, column);
}
- throw new TableNotFoundException(schemaTableName);
}
private CassandraColumnHandle buildColumnHandle(TableMetadata tableMetadata, ColumnMetadata columnMeta, boolean partitionKey, boolean clusteringKey, int ordinalPosition, boolean hidden)
@@ -297,7 +388,25 @@ public List getPartitions(CassandraTable table, List
return partitions.build();
}
- protected Iterable queryPartitionKeys(CassandraTable table, List filterPrefix)
+ @Override
+ public ResultSet execute(String cql, Object... values)
+ {
+ return executeWithSession(session -> session.execute(cql, values));
+ }
+
+ @Override
+ public PreparedStatement prepare(RegularStatement statement)
+ {
+ return executeWithSession(session -> session.prepare(statement));
+ }
+
+ @Override
+ public ResultSet execute(Statement statement)
+ {
+ return executeWithSession(session -> session.execute(statement));
+ }
+
+ private Iterable queryPartitionKeys(CassandraTable table, List filterPrefix)
{
CassandraTableHandle tableHandle = table.getTableHandle();
List partitionKeyColumns = table.getPartitionKeyColumns();
@@ -308,11 +417,53 @@ protected Iterable queryPartitionKeys(CassandraTable table, List fi
Select partitionKeys = CassandraCqlUtils.selectDistinctFrom(tableHandle, partitionKeyColumns);
addWhereClause(partitionKeys.where(), partitionKeyColumns, filterPrefix);
- return executeWithSession(session -> session.execute(partitionKeys)).all();
+ return execute(partitionKeys).all();
+ }
+
+ private static void addWhereClause(Where where, List partitionKeyColumns, List filterPrefix)
+ {
+ for (int i = 0; i < filterPrefix.size(); i++) {
+ CassandraColumnHandle column = partitionKeyColumns.get(i);
+ Object value = column.getCassandraType().getJavaValue(filterPrefix.get(i));
+ Clause clause = QueryBuilder.eq(CassandraCqlUtils.validColumnName(column.getName()), value);
+ where.and(clause);
+ }
}
@Override
- public T executeWithSession(SessionCallable sessionCallable)
+ public List getSizeEstimates(String keyspaceName, String tableName)
+ {
+ checkSizeEstimatesTableExist();
+ Statement statement = select("range_start", "range_end", "mean_partition_size", "partitions_count")
+ .from(SYSTEM, SIZE_ESTIMATES)
+ .where(eq("keyspace_name", keyspaceName))
+ .and(eq("table_name", tableName));
+
+ ResultSet result = executeWithSession(session -> session.execute(statement));
+ ImmutableList.Builder estimates = ImmutableList.builder();
+ for (Row row : result.all()) {
+ SizeEstimate estimate = new SizeEstimate(
+ row.getString("range_start"),
+ row.getString("range_end"),
+ row.getLong("mean_partition_size"),
+ row.getLong("partitions_count"));
+ estimates.add(estimate);
+ }
+
+ return estimates.build();
+ }
+
+ private void checkSizeEstimatesTableExist()
+ {
+ KeyspaceMetadata keyspaceMetadata = executeWithSession(session -> session.getCluster().getMetadata().getKeyspace(SYSTEM));
+ checkState(keyspaceMetadata != null, "system keyspace metadata must not be null");
+ TableMetadata table = keyspaceMetadata.getTable(SIZE_ESTIMATES);
+ if (table == null) {
+ throw new PrestoException(NOT_SUPPORTED, "Cassandra versions prior to 2.1.5 are not supported");
+ }
+ }
+
+ private T executeWithSession(SessionCallable sessionCallable)
{
ReconnectionPolicy reconnectionPolicy = cluster.getConfiguration().getPolicies().getReconnectionPolicy();
ReconnectionSchedule schedule = reconnectionPolicy.newSchedule();
@@ -342,13 +493,8 @@ public T executeWithSession(SessionCallable sessionCallable)
}
}
- private static void addWhereClause(Where where, List partitionKeyColumns, List filterPrefix)
+ private interface SessionCallable
{
- for (int i = 0; i < filterPrefix.size(); i++) {
- CassandraColumnHandle column = partitionKeyColumns.get(i);
- Object value = column.getCassandraType().getJavaValue(filterPrefix.get(i));
- Clause clause = QueryBuilder.eq(CassandraCqlUtils.validColumnName(column.getName()), value);
- where.and(clause);
- }
+ T executeWithSession(Session session);
}
}
diff --git a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/ReopeningCluster.java b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/ReopeningCluster.java
new file mode 100644
index 0000000000000..a8c988b460ed4
--- /dev/null
+++ b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/ReopeningCluster.java
@@ -0,0 +1,88 @@
+/*
+ * 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.cassandra;
+
+import com.datastax.driver.core.CloseFuture;
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.DelegatingCluster;
+import io.airlift.log.Logger;
+
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.util.function.Supplier;
+
+import static com.google.common.base.Preconditions.checkState;
+import static com.google.common.base.Verify.verify;
+import static java.util.Objects.requireNonNull;
+
+@ThreadSafe
+public class ReopeningCluster
+ extends DelegatingCluster
+{
+ private static final Logger log = Logger.get(ReopeningCluster.class);
+
+ @GuardedBy("this")
+ private Cluster delegate;
+ @GuardedBy("this")
+ private boolean closed;
+
+ private final Supplier supplier;
+
+ public ReopeningCluster(Supplier supplier)
+ {
+ this.supplier = requireNonNull(supplier, "supplier is null");
+ }
+
+ @Override
+ protected synchronized Cluster delegate()
+ {
+ checkState(!closed, "Cluster has been closed");
+
+ if (delegate == null) {
+ delegate = supplier.get();
+ }
+
+ if (delegate.isClosed()) {
+ log.warn("Cluster has been closed internally");
+ delegate = supplier.get();
+ }
+
+ verify(!delegate.isClosed(), "Newly created cluster has been immediately closed");
+
+ return delegate;
+ }
+
+ @Override
+ public synchronized void close()
+ {
+ closed = true;
+ if (delegate != null) {
+ delegate.close();
+ delegate = null;
+ }
+ }
+
+ @Override
+ public synchronized boolean isClosed()
+ {
+ return closed;
+ }
+
+ @Override
+ public synchronized CloseFuture closeAsync()
+ {
+ throw new UnsupportedOperationException();
+ }
+}
diff --git a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/SizeEstimate.java b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/SizeEstimate.java
new file mode 100644
index 0000000000000..5779f9ffe18dd
--- /dev/null
+++ b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/SizeEstimate.java
@@ -0,0 +1,88 @@
+/*
+ * 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.cassandra;
+
+import java.util.Objects;
+
+import static com.google.common.base.MoreObjects.toStringHelper;
+import static java.util.Objects.requireNonNull;
+
+public class SizeEstimate
+{
+ private final String rangeStart;
+ private final String rangeEnd;
+ private final long meanPartitionSize;
+ private final long partitionsCount;
+
+ public SizeEstimate(String rangeStart, String rangeEnd, long meanPartitionSize, long partitionsCount)
+ {
+ this.rangeStart = requireNonNull(rangeStart, "rangeStart is null");
+ this.rangeEnd = requireNonNull(rangeEnd, "rangeEnd is null");
+ this.meanPartitionSize = meanPartitionSize;
+ this.partitionsCount = partitionsCount;
+ }
+
+ public String getRangeStart()
+ {
+ return rangeStart;
+ }
+
+ public String getRangeEnd()
+ {
+ return rangeEnd;
+ }
+
+ public long getMeanPartitionSize()
+ {
+ return meanPartitionSize;
+ }
+
+ public long getPartitionsCount()
+ {
+ return partitionsCount;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ SizeEstimate that = (SizeEstimate) o;
+ return meanPartitionSize == that.meanPartitionSize &&
+ partitionsCount == that.partitionsCount &&
+ Objects.equals(rangeStart, that.rangeStart) &&
+ Objects.equals(rangeEnd, that.rangeEnd);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(rangeStart, rangeEnd, meanPartitionSize, partitionsCount);
+ }
+
+ @Override
+ public String toString()
+ {
+ return toStringHelper(this)
+ .add("rangeStart", rangeStart)
+ .add("rangeEnd", rangeEnd)
+ .add("meanPartitionSize", meanPartitionSize)
+ .add("partitionsCount", partitionsCount)
+ .toString();
+ }
+}
diff --git a/presto-cassandra/src/test/java/com/facebook/presto/cassandra/CassandraQueryRunner.java b/presto-cassandra/src/test/java/com/facebook/presto/cassandra/CassandraQueryRunner.java
index 0f95f82a1f429..8dbba13e5b1c0 100644
--- a/presto-cassandra/src/test/java/com/facebook/presto/cassandra/CassandraQueryRunner.java
+++ b/presto-cassandra/src/test/java/com/facebook/presto/cassandra/CassandraQueryRunner.java
@@ -55,9 +55,8 @@ public static synchronized DistributedQueryRunner createCassandraQueryRunner()
List> tables = TpchTable.getTables();
copyTpchTables(queryRunner, "tpch", TINY_SCHEMA_NAME, createCassandraSession("tpch"), tables);
for (TpchTable table : tables) {
- EmbeddedCassandra.flush("tpch", table.getTableName());
+ EmbeddedCassandra.refreshSizeEstimates("tpch", table.getTableName());
}
- EmbeddedCassandra.refreshSizeEstimates();
tpchLoaded = true;
}
diff --git a/presto-cassandra/src/test/java/com/facebook/presto/cassandra/CassandraTestingUtils.java b/presto-cassandra/src/test/java/com/facebook/presto/cassandra/CassandraTestingUtils.java
index 936f84946e1f1..26334b8434fca 100644
--- a/presto-cassandra/src/test/java/com/facebook/presto/cassandra/CassandraTestingUtils.java
+++ b/presto-cassandra/src/test/java/com/facebook/presto/cassandra/CassandraTestingUtils.java
@@ -33,21 +33,25 @@
public class CassandraTestingUtils
{
public static final String TABLE_ALL_TYPES = "table_all_types";
+ public static final String TABLE_ALL_TYPES_INSERT = "table_all_types_insert";
public static final String TABLE_ALL_TYPES_PARTITION_KEY = "table_all_types_partition_key";
public static final String TABLE_CLUSTERING_KEYS = "table_clustering_keys";
public static final String TABLE_CLUSTERING_KEYS_LARGE = "table_clustering_keys_large";
public static final String TABLE_MULTI_PARTITION_CLUSTERING_KEYS = "table_multi_partition_clustering_keys";
+ public static final String TABLE_CLUSTERING_KEYS_INEQUALITY = "table_clustering_keys_inequality";
private CassandraTestingUtils() {}
public static void createTestTables(CassandraSession cassandraSession, String keyspace, Date date)
{
createKeyspace(cassandraSession, keyspace);
- createTableAllTypes(cassandraSession, new SchemaTableName(keyspace, TABLE_ALL_TYPES), date);
+ createTableAllTypes(cassandraSession, new SchemaTableName(keyspace, TABLE_ALL_TYPES), date, 9);
+ createTableAllTypes(cassandraSession, new SchemaTableName(keyspace, TABLE_ALL_TYPES_INSERT), date, 0);
createTableAllTypesPartitionKey(cassandraSession, new SchemaTableName(keyspace, TABLE_ALL_TYPES_PARTITION_KEY), date);
createTableClusteringKeys(cassandraSession, new SchemaTableName(keyspace, TABLE_CLUSTERING_KEYS), 9);
createTableClusteringKeys(cassandraSession, new SchemaTableName(keyspace, TABLE_CLUSTERING_KEYS_LARGE), 1000);
createTableMultiPartitionClusteringKeys(cassandraSession, new SchemaTableName(keyspace, TABLE_MULTI_PARTITION_CLUSTERING_KEYS));
+ createTableClusteringKeysInequality(cassandraSession, new SchemaTableName(keyspace, TABLE_CLUSTERING_KEYS_INEQUALITY), date, 4);
}
public static void createKeyspace(CassandraSession session, String keyspaceName)
@@ -77,7 +81,7 @@ public static void insertIntoTableClusteringKeys(CassandraSession session, Schem
.value("clust_one", "clust_one")
.value("clust_two", "clust_two_" + rowNumber.toString())
.value("clust_three", "clust_three_" + rowNumber.toString());
- session.executeWithSession(s -> s.execute(insert));
+ session.execute(insert);
}
assertEquals(session.execute("SELECT COUNT(*) FROM " + table).all().get(0).getLong(0), rowsCount);
}
@@ -106,12 +110,39 @@ public static void insertIntoTableMultiPartitionClusteringKeys(CassandraSession
.value("clust_one", "clust_one")
.value("clust_two", "clust_two_" + rowNumber.toString())
.value("clust_three", "clust_three_" + rowNumber.toString());
- session.executeWithSession(s -> s.execute(insert));
+ session.execute(insert);
}
assertEquals(session.execute("SELECT COUNT(*) FROM " + table).all().get(0).getLong(0), 9);
}
- public static void createTableAllTypes(CassandraSession session, SchemaTableName table, Date date)
+ public static void createTableClusteringKeysInequality(CassandraSession session, SchemaTableName table, Date date, int rowsCount)
+ {
+ session.execute("DROP TABLE IF EXISTS " + table);
+ session.execute("CREATE TABLE " + table + " (" +
+ "key text, " +
+ "clust_one text, " +
+ "clust_two int, " +
+ "clust_three timestamp, " +
+ "data text, " +
+ "PRIMARY KEY((key), clust_one, clust_two, clust_three) " +
+ ")");
+ insertIntoTableClusteringKeysInequality(session, table, date, rowsCount);
+ }
+
+ public static void insertIntoTableClusteringKeysInequality(CassandraSession session, SchemaTableName table, Date date, int rowsCount)
+ {
+ for (Integer rowNumber = 1; rowNumber <= rowsCount; rowNumber++) {
+ Insert insert = QueryBuilder.insertInto(table.getSchemaName(), table.getTableName())
+ .value("key", "key_1")
+ .value("clust_one", "clust_one")
+ .value("clust_two", rowNumber)
+ .value("clust_three", date.getTime() + rowNumber * 10);
+ session.execute(insert);
+ }
+ assertEquals(session.execute("SELECT COUNT(*) FROM " + table).all().get(0).getLong(0), rowsCount);
+ }
+
+ public static void createTableAllTypes(CassandraSession session, SchemaTableName table, Date date, int rowsCount)
{
session.execute("DROP TABLE IF EXISTS " + table);
session.execute("CREATE TABLE " + table + " (" +
@@ -134,7 +165,7 @@ public static void createTableAllTypes(CassandraSession session, SchemaTableName
" typemap map, " +
" typeset set, " +
")");
- insertTestData(session, table, date);
+ insertTestData(session, table, date, rowsCount);
}
public static void createTableAllTypesPartitionKey(CassandraSession session, SchemaTableName table, Date date)
@@ -186,12 +217,12 @@ public static void createTableAllTypesPartitionKey(CassandraSession session, Sch
" ))" +
")");
- insertTestData(session, table, date);
+ insertTestData(session, table, date, 9);
}
- private static void insertTestData(CassandraSession session, SchemaTableName table, Date date)
+ private static void insertTestData(CassandraSession session, SchemaTableName table, Date date, int rowsCount)
{
- for (Integer rowNumber = 1; rowNumber < 10; rowNumber++) {
+ for (Integer rowNumber = 1; rowNumber <= rowsCount; rowNumber++) {
Insert insert = QueryBuilder.insertInto(table.getSchemaName(), table.getTableName())
.value("key", "key " + rowNumber.toString())
.value("typeuuid", UUID.fromString(String.format("00000000-0000-0000-0000-%012d", rowNumber)))
@@ -212,8 +243,8 @@ private static void insertTestData(CassandraSession session, SchemaTableName tab
.value("typemap", ImmutableMap.of(rowNumber, rowNumber + 1L, rowNumber + 2, rowNumber + 3L))
.value("typeset", ImmutableSet.of(false, true));
- session.executeWithSession(s -> s.execute(insert));
+ session.execute(insert);
}
- assertEquals(session.execute("SELECT COUNT(*) FROM " + table).all().get(0).getLong(0), 9);
+ assertEquals(session.execute("SELECT COUNT(*) FROM " + table).all().get(0).getLong(0), rowsCount);
}
}
diff --git a/presto-cassandra/src/test/java/com/facebook/presto/cassandra/EmbeddedCassandra.java b/presto-cassandra/src/test/java/com/facebook/presto/cassandra/EmbeddedCassandra.java
index 0b3c4c3bd7cb0..dd6b0f9e9f61b 100644
--- a/presto-cassandra/src/test/java/com/facebook/presto/cassandra/EmbeddedCassandra.java
+++ b/presto-cassandra/src/test/java/com/facebook/presto/cassandra/EmbeddedCassandra.java
@@ -32,15 +32,18 @@
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.List;
+import java.util.concurrent.TimeoutException;
import static com.datastax.driver.core.ProtocolVersion.V3;
import static com.google.common.base.Preconditions.checkState;
import static com.google.common.io.Files.createTempDir;
import static com.google.common.io.Files.write;
import static com.google.common.io.Resources.getResource;
+import static java.lang.String.format;
import static java.nio.charset.StandardCharsets.UTF_8;
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.TimeUnit.MINUTES;
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
import static org.testng.Assert.assertEquals;
public final class EmbeddedCassandra
@@ -50,6 +53,8 @@ public final class EmbeddedCassandra
private static final String HOST = "127.0.0.1";
private static final int PORT = 9142;
+ private static final Duration REFRESH_SIZE_ESTIMATES_TIMEOUT = new Duration(1, MINUTES);
+
private static CassandraSession session;
private static boolean initialized;
@@ -76,6 +81,7 @@ public static synchronized void start()
.withClusterName("TestCluster")
.addContactPointsWithPorts(ImmutableList.of(
new InetSocketAddress(HOST, PORT)))
+ .withMaxSchemaAgreementWaitSeconds(30)
.build();
CassandraSession session = new NativeCassandraSession(
@@ -148,7 +154,24 @@ private static void checkConnectivity(CassandraSession session)
log.info("Cassandra version: %s", version);
}
- public static void flush(String keyspace, String table)
+ public static void refreshSizeEstimates(String keyspace, String table)
+ throws Exception
+ {
+ long deadline = System.nanoTime() + REFRESH_SIZE_ESTIMATES_TIMEOUT.roundTo(NANOSECONDS);
+ while (System.nanoTime() < deadline) {
+ flushTable(keyspace, table);
+ refreshSizeEstimates();
+ List sizeEstimates = getSession().getSizeEstimates(keyspace, table);
+ if (!sizeEstimates.isEmpty()) {
+ log.info("Size estimates for the table %s.%s have been refreshed successfully: %s", keyspace, table, sizeEstimates);
+ return;
+ }
+ log.info("Size estimates haven't been refreshed as expected. Retrying ...");
+ }
+ throw new TimeoutException(format("Attempting to refresh size estimates for table %s.%s has timed out after %s", keyspace, table, REFRESH_SIZE_ESTIMATES_TIMEOUT));
+ }
+
+ private static void flushTable(String keyspace, String table)
throws Exception
{
ManagementFactory
@@ -160,7 +183,7 @@ public static void flush(String keyspace, String table)
new String[] {"java.lang.String", "[Ljava.lang.String;"});
}
- public static void refreshSizeEstimates()
+ private static void refreshSizeEstimates()
throws Exception
{
ManagementFactory
diff --git a/presto-cassandra/src/test/java/com/facebook/presto/cassandra/MockCassandraSession.java b/presto-cassandra/src/test/java/com/facebook/presto/cassandra/MockCassandraSession.java
deleted file mode 100644
index 3e4365d3f2bee..0000000000000
--- a/presto-cassandra/src/test/java/com/facebook/presto/cassandra/MockCassandraSession.java
+++ /dev/null
@@ -1,133 +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.cassandra;
-
-import com.datastax.driver.core.Host;
-import com.facebook.presto.spi.SchemaNotFoundException;
-import com.facebook.presto.spi.SchemaTableName;
-import com.facebook.presto.spi.TableNotFoundException;
-import com.google.common.collect.ImmutableList;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import static java.util.Objects.requireNonNull;
-
-public class MockCassandraSession
- implements CassandraSession
-{
- static final String TEST_SCHEMA = "testkeyspace";
- static final String BAD_SCHEMA = "badkeyspace";
- static final String TEST_TABLE = "testtbl";
- static final String TEST_COLUMN1 = "column1";
- static final String TEST_COLUMN2 = "column2";
-
- private final AtomicInteger accessCount = new AtomicInteger();
- private boolean throwException;
- private final String connectorId;
-
- public MockCassandraSession(String connectorId)
- {
- this.connectorId = requireNonNull(connectorId, "connectorId is null");
- }
-
- public void setThrowException(boolean throwException)
- {
- this.throwException = throwException;
- }
-
- public int getAccessCount()
- {
- return accessCount.get();
- }
-
- @Override
- public List getAllSchemas()
- {
- accessCount.incrementAndGet();
-
- if (throwException) {
- throw new IllegalStateException();
- }
- return ImmutableList.of(TEST_SCHEMA);
- }
-
- @Override
- public List getAllTables(String schema)
- throws SchemaNotFoundException
- {
- accessCount.incrementAndGet();
- if (throwException) {
- throw new IllegalStateException();
- }
-
- if (schema.equals(TEST_SCHEMA)) {
- return ImmutableList.of(TEST_TABLE);
- }
- throw new SchemaNotFoundException(schema);
- }
-
- @Override
- public void getSchema(String schema)
- throws SchemaNotFoundException
- {
- accessCount.incrementAndGet();
- if (throwException) {
- throw new IllegalStateException();
- }
-
- if (!schema.equals(TEST_SCHEMA)) {
- throw new SchemaNotFoundException(schema);
- }
- }
-
- @Override
- public CassandraTable getTable(SchemaTableName tableName)
- throws TableNotFoundException
- {
- accessCount.incrementAndGet();
- if (throwException) {
- throw new IllegalStateException();
- }
-
- if (tableName.getSchemaName().equals(TEST_SCHEMA) && tableName.getTableName().equals(TEST_TABLE)) {
- return new CassandraTable(
- new CassandraTableHandle(connectorId, TEST_SCHEMA, TEST_TABLE),
- ImmutableList.of(
- new CassandraColumnHandle(connectorId, TEST_COLUMN1, 0, CassandraType.VARCHAR, null, true, false, false, false),
- new CassandraColumnHandle(connectorId, TEST_COLUMN2, 0, CassandraType.INT, null, false, false, false, false)));
- }
- throw new TableNotFoundException(tableName);
- }
-
- @Override
- public List getPartitions(CassandraTable table, List filterPrefix)
- {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public T executeWithSession(SessionCallable sessionCallable)
- {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public Set getReplicas(String schemaName, ByteBuffer partitionKey)
- {
- throw new UnsupportedOperationException();
- }
-}
diff --git a/presto-cassandra/src/test/java/com/facebook/presto/cassandra/TestCachingCassandraSchemaProvider.java b/presto-cassandra/src/test/java/com/facebook/presto/cassandra/TestCachingCassandraSchemaProvider.java
deleted file mode 100644
index 1f1c8c81b3004..0000000000000
--- a/presto-cassandra/src/test/java/com/facebook/presto/cassandra/TestCachingCassandraSchemaProvider.java
+++ /dev/null
@@ -1,141 +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.cassandra;
-
-import com.facebook.presto.spi.SchemaNotFoundException;
-import com.facebook.presto.spi.TableNotFoundException;
-import com.google.common.collect.ImmutableList;
-import com.google.common.util.concurrent.ListeningExecutorService;
-import io.airlift.units.Duration;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Test;
-
-import java.util.concurrent.TimeUnit;
-
-import static com.facebook.presto.cassandra.MockCassandraSession.BAD_SCHEMA;
-import static com.facebook.presto.cassandra.MockCassandraSession.TEST_SCHEMA;
-import static com.facebook.presto.cassandra.MockCassandraSession.TEST_TABLE;
-import static com.google.common.util.concurrent.MoreExecutors.listeningDecorator;
-import static io.airlift.concurrent.Threads.daemonThreadsNamed;
-import static java.util.concurrent.Executors.newCachedThreadPool;
-import static org.testng.Assert.assertEquals;
-import static org.testng.Assert.assertNotNull;
-
-@Test(singleThreaded = true)
-public class TestCachingCassandraSchemaProvider
-{
- private static final String CONNECTOR_ID = "test-cassandra";
- private MockCassandraSession mockSession;
- private CachingCassandraSchemaProvider schemaProvider;
-
- @BeforeMethod
- public void setUp()
- throws Exception
- {
- mockSession = new MockCassandraSession(CONNECTOR_ID);
- ListeningExecutorService executor = listeningDecorator(newCachedThreadPool(daemonThreadsNamed("test-%s")));
- schemaProvider = new CachingCassandraSchemaProvider(
- CONNECTOR_ID,
- mockSession,
- executor,
- new Duration(5, TimeUnit.MINUTES),
- new Duration(1, TimeUnit.MINUTES));
- }
-
- @Test
- public void testGetAllDatabases()
- throws Exception
- {
- assertEquals(mockSession.getAccessCount(), 0);
- assertEquals(schemaProvider.getAllSchemas(), ImmutableList.of(TEST_SCHEMA));
- assertEquals(mockSession.getAccessCount(), 1);
- assertEquals(schemaProvider.getAllSchemas(), ImmutableList.of(TEST_SCHEMA));
- assertEquals(mockSession.getAccessCount(), 1);
-
- schemaProvider.flushCache();
-
- assertEquals(schemaProvider.getAllSchemas(), ImmutableList.of(TEST_SCHEMA));
- assertEquals(mockSession.getAccessCount(), 2);
- }
-
- @Test
- public void testGetAllTable()
- throws Exception
- {
- assertEquals(mockSession.getAccessCount(), 0);
- assertEquals(schemaProvider.getAllTables(TEST_SCHEMA), ImmutableList.of(TEST_TABLE));
- assertEquals(mockSession.getAccessCount(), 2);
- assertEquals(schemaProvider.getAllTables(TEST_SCHEMA), ImmutableList.of(TEST_TABLE));
- assertEquals(mockSession.getAccessCount(), 2);
-
- schemaProvider.flushCache();
-
- assertEquals(schemaProvider.getAllTables(TEST_SCHEMA), ImmutableList.of(TEST_TABLE));
- assertEquals(mockSession.getAccessCount(), 4);
- }
-
- @Test(expectedExceptions = SchemaNotFoundException.class)
- public void testInvalidDbGetAllTAbles()
- throws Exception
- {
- schemaProvider.getAllTables(BAD_SCHEMA);
- }
-
- @Test
- public void testGetTable()
- throws Exception
- {
- CassandraTableHandle tableHandle = new CassandraTableHandle(CONNECTOR_ID, TEST_SCHEMA, TEST_TABLE);
- assertEquals(mockSession.getAccessCount(), 0);
- assertNotNull(schemaProvider.getTable(tableHandle));
- assertEquals(mockSession.getAccessCount(), 1);
- assertNotNull(schemaProvider.getTable(tableHandle));
- assertEquals(mockSession.getAccessCount(), 1);
-
- schemaProvider.flushCache();
-
- assertNotNull(schemaProvider.getTable(tableHandle));
- assertEquals(mockSession.getAccessCount(), 2);
- }
-
- @Test(expectedExceptions = TableNotFoundException.class)
- public void testInvalidDbGetTable()
- throws Exception
- {
- CassandraTableHandle tableHandle = new CassandraTableHandle(CONNECTOR_ID, BAD_SCHEMA, TEST_TABLE);
- schemaProvider.getTable(tableHandle);
- }
-
- @Test
- public void testNoCacheExceptions()
- throws Exception
- {
- // Throw exceptions on usage
- mockSession.setThrowException(true);
- try {
- schemaProvider.getAllSchemas();
- }
- catch (RuntimeException ignored) {
- }
- assertEquals(mockSession.getAccessCount(), 1);
-
- // Second try should hit the client again
- try {
- schemaProvider.getAllSchemas();
- }
- catch (RuntimeException ignored) {
- }
- assertEquals(mockSession.getAccessCount(), 2);
- }
-}
diff --git a/presto-cassandra/src/test/java/com/facebook/presto/cassandra/TestCassandraClientConfig.java b/presto-cassandra/src/test/java/com/facebook/presto/cassandra/TestCassandraClientConfig.java
index 28da5db5591f4..7d9503f355b10 100644
--- a/presto-cassandra/src/test/java/com/facebook/presto/cassandra/TestCassandraClientConfig.java
+++ b/presto-cassandra/src/test/java/com/facebook/presto/cassandra/TestCassandraClientConfig.java
@@ -21,7 +21,6 @@
import org.testng.annotations.Test;
import java.util.Map;
-import java.util.concurrent.TimeUnit;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.MINUTES;
@@ -33,9 +32,6 @@ public class TestCassandraClientConfig
public void testDefaults()
{
ConfigAssertions.assertRecordedDefaults(ConfigAssertions.recordDefaults(CassandraClientConfig.class)
- .setMaxSchemaRefreshThreads(1)
- .setSchemaCacheTtl(new Duration(1, TimeUnit.HOURS))
- .setSchemaRefreshInterval(new Duration(2, TimeUnit.MINUTES))
.setFetchSize(5_000)
.setConsistencyLevel(ConsistencyLevel.ONE)
.setContactPoints("")
@@ -66,9 +62,6 @@ public void testDefaults()
public void testExplicitPropertyMappings()
{
Map properties = new ImmutableMap.Builder()
- .put("cassandra.max-schema-refresh-threads", "2")
- .put("cassandra.schema-cache-ttl", "2h")
- .put("cassandra.schema-refresh-interval", "30m")
.put("cassandra.contact-points", "host1,host2")
.put("cassandra.native-protocol-port", "9999")
.put("cassandra.fetch-size", "10000")
@@ -96,9 +89,6 @@ public void testExplicitPropertyMappings()
.build();
CassandraClientConfig expected = new CassandraClientConfig()
- .setMaxSchemaRefreshThreads(2)
- .setSchemaCacheTtl(new Duration(2, TimeUnit.HOURS))
- .setSchemaRefreshInterval(new Duration(30, TimeUnit.MINUTES))
.setContactPoints("host1", "host2")
.setNativeProtocolPort(9999)
.setFetchSize(10_000)
diff --git a/presto-cassandra/src/test/java/com/facebook/presto/cassandra/TestCassandraIntegrationSmokeTest.java b/presto-cassandra/src/test/java/com/facebook/presto/cassandra/TestCassandraIntegrationSmokeTest.java
index 1efa56d1d59a6..7506ceffc03e1 100644
--- a/presto-cassandra/src/test/java/com/facebook/presto/cassandra/TestCassandraIntegrationSmokeTest.java
+++ b/presto-cassandra/src/test/java/com/facebook/presto/cassandra/TestCassandraIntegrationSmokeTest.java
@@ -19,6 +19,7 @@
import com.facebook.presto.testing.MaterializedRow;
import com.facebook.presto.tests.AbstractTestIntegrationSmokeTest;
import com.google.common.collect.ImmutableList;
+import io.airlift.units.Duration;
import org.joda.time.DateTime;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;
@@ -32,8 +33,10 @@
import static com.datastax.driver.core.utils.Bytes.toRawHexString;
import static com.facebook.presto.cassandra.CassandraQueryRunner.createCassandraSession;
import static com.facebook.presto.cassandra.CassandraTestingUtils.TABLE_ALL_TYPES;
+import static com.facebook.presto.cassandra.CassandraTestingUtils.TABLE_ALL_TYPES_INSERT;
import static com.facebook.presto.cassandra.CassandraTestingUtils.TABLE_ALL_TYPES_PARTITION_KEY;
import static com.facebook.presto.cassandra.CassandraTestingUtils.TABLE_CLUSTERING_KEYS;
+import static com.facebook.presto.cassandra.CassandraTestingUtils.TABLE_CLUSTERING_KEYS_INEQUALITY;
import static com.facebook.presto.cassandra.CassandraTestingUtils.TABLE_CLUSTERING_KEYS_LARGE;
import static com.facebook.presto.cassandra.CassandraTestingUtils.TABLE_MULTI_PARTITION_CLUSTERING_KEYS;
import static com.facebook.presto.cassandra.CassandraTestingUtils.createTestTables;
@@ -47,7 +50,11 @@
import static com.facebook.presto.spi.type.VarcharType.createUnboundedVarcharType;
import static com.facebook.presto.spi.type.VarcharType.createVarcharType;
import static com.facebook.presto.testing.MaterializedResult.DEFAULT_PRECISION;
+import static com.facebook.presto.testing.MaterializedResult.resultBuilder;
+import static com.facebook.presto.tests.QueryAssertions.assertContains;
+import static com.facebook.presto.tests.QueryAssertions.assertContainsEventually;
import static com.google.common.primitives.Ints.toByteArray;
+import static java.util.concurrent.TimeUnit.MINUTES;
import static java.util.stream.Collectors.toList;
import static org.joda.time.DateTimeZone.UTC;
import static org.testng.Assert.assertEquals;
@@ -63,6 +70,8 @@ public class TestCassandraIntegrationSmokeTest
private static final Date DATE_LOCAL = new Date(DATE_TIME_UTC.getMillis());
private static final Timestamp TIMESTAMP_LOCAL = new Timestamp(DATE_TIME_UTC.getMillis());
+ private CassandraSession session;
+
public TestCassandraIntegrationSmokeTest()
throws Exception
{
@@ -73,7 +82,8 @@ public TestCassandraIntegrationSmokeTest()
public void setUp()
throws Exception
{
- createTestTables(EmbeddedCassandra.getSession(), KEYSPACE, DATE_LOCAL);
+ session = EmbeddedCassandra.getSession();
+ createTestTables(session, KEYSPACE, DATE_LOCAL);
}
@Test
@@ -193,6 +203,304 @@ public void testClusteringKeyOnlyPushdown()
assertEquals(execute(sql).getRowCount(), 1);
sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_LARGE + " WHERE clust_one='clust_one' AND clust_two='clust_two_2' AND clust_three='clust_three_2'";
assertEquals(execute(sql).getRowCount(), 1);
+ sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_LARGE + " WHERE clust_one='clust_one' AND clust_two='clust_two_2' AND clust_three IN ('clust_three_1', 'clust_three_2', 'clust_three_3')";
+ assertEquals(execute(sql).getRowCount(), 1);
+ sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_LARGE + " WHERE clust_one='clust_one' AND clust_two IN ('clust_two_1','clust_two_2') AND clust_three IN ('clust_three_1', 'clust_three_2', 'clust_three_3')";
+ assertEquals(execute(sql).getRowCount(), 2);
+ sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_LARGE + " WHERE clust_one='clust_one' AND clust_two > 'clust_two_998'";
+ assertEquals(execute(sql).getRowCount(), 1);
+ sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_LARGE + " WHERE clust_one='clust_one' AND clust_two > 'clust_two_997' AND clust_two < 'clust_two_999'";
+ assertEquals(execute(sql).getRowCount(), 1);
+ sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_LARGE + " WHERE clust_one='clust_one' AND clust_two IN ('clust_two_1','clust_two_2') AND clust_three > 'clust_three_998'";
+ assertEquals(execute(sql).getRowCount(), 0);
+ sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_LARGE + " WHERE clust_one='clust_one' AND clust_two IN ('clust_two_1','clust_two_2') AND clust_three < 'clust_three_3'";
+ assertEquals(execute(sql).getRowCount(), 2);
+ sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_LARGE + " WHERE clust_one='clust_one' AND clust_two IN ('clust_two_1','clust_two_2') AND clust_three > 'clust_three_1' AND clust_three < 'clust_three_3'";
+ assertEquals(execute(sql).getRowCount(), 1);
+ sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_LARGE + " WHERE clust_one='clust_one' AND clust_two IN ('clust_two_1','clust_two_2','clust_two_3') AND clust_two < 'clust_two_2'";
+ assertEquals(execute(sql).getRowCount(), 1);
+ sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_LARGE + " WHERE clust_one='clust_one' AND clust_two IN ('clust_two_997','clust_two_998','clust_two_999') AND clust_two > 'clust_two_998'";
+ assertEquals(execute(sql).getRowCount(), 1);
+ sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_LARGE + " WHERE clust_one='clust_one' AND clust_two IN ('clust_two_1','clust_two_2','clust_two_3') AND clust_two = 'clust_two_2'";
+ assertEquals(execute(sql).getRowCount(), 1);
+ }
+
+ @Test
+ public void testClusteringKeyPushdownInequality()
+ throws Exception
+ {
+ String sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_INEQUALITY + " WHERE key='key_1' AND clust_one='clust_one'";
+ assertEquals(execute(sql).getRowCount(), 4);
+ sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_INEQUALITY + " WHERE key='key_1' AND clust_one='clust_one' AND clust_two=2";
+ assertEquals(execute(sql).getRowCount(), 1);
+ sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_INEQUALITY + " WHERE key='key_1' AND clust_one='clust_one' AND clust_two=2 AND clust_three = timestamp '1970-01-01 03:04:05.020'";
+ assertEquals(execute(sql).getRowCount(), 1);
+ sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_INEQUALITY + " WHERE key='key_1' AND clust_one='clust_one' AND clust_two=2 AND clust_three = timestamp '1970-01-01 03:04:05.010'";
+ assertEquals(execute(sql).getRowCount(), 0);
+ sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_INEQUALITY + " WHERE key='key_1' AND clust_one='clust_one' AND clust_two IN (1,2)";
+ assertEquals(execute(sql).getRowCount(), 2);
+ sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_INEQUALITY + " WHERE key='key_1' AND clust_one='clust_one' AND clust_two > 1 AND clust_two < 3";
+ assertEquals(execute(sql).getRowCount(), 1);
+ sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_INEQUALITY + " WHERE key='key_1' AND clust_one='clust_one' AND clust_two=2 AND clust_three >= timestamp '1970-01-01 03:04:05.010' AND clust_three <= timestamp '1970-01-01 03:04:05.020'";
+ assertEquals(execute(sql).getRowCount(), 1);
+ sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_INEQUALITY + " WHERE key='key_1' AND clust_one='clust_one' AND clust_two IN (1,2) AND clust_three >= timestamp '1970-01-01 03:04:05.010' AND clust_three <= timestamp '1970-01-01 03:04:05.020'";
+ assertEquals(execute(sql).getRowCount(), 2);
+ sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_INEQUALITY + " WHERE key='key_1' AND clust_one='clust_one' AND clust_two IN (1,2,3) AND clust_two < 2";
+ assertEquals(execute(sql).getRowCount(), 1);
+ sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_INEQUALITY + " WHERE key='key_1' AND clust_one='clust_one' AND clust_two IN (1,2,3) AND clust_two > 2";
+ assertEquals(execute(sql).getRowCount(), 1);
+ sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_INEQUALITY + " WHERE key='key_1' AND clust_one='clust_one' AND clust_two IN (1,2,3) AND clust_two = 2";
+ assertEquals(execute(sql).getRowCount(), 1);
+ }
+
+ @Test
+ public void testUpperCaseNameUnescapedInCassandra()
+ throws Exception
+ {
+ /*
+ * If an identifier is not escaped with double quotes it is stored as lowercase in the Cassandra metadata
+ *
+ * http://docs.datastax.com/en/cql/3.1/cql/cql_reference/ucase-lcase_r.html
+ */
+ session.execute("CREATE KEYSPACE KEYSPACE_1 WITH REPLICATION = {'class':'SimpleStrategy', 'replication_factor': 1}");
+ assertContainsEventually(() -> execute("SHOW SCHEMAS FROM cassandra"), resultBuilder(getSession(), createUnboundedVarcharType())
+ .row("keyspace_1")
+ .build(), new Duration(1, MINUTES));
+
+ session.execute("CREATE TABLE KEYSPACE_1.TABLE_1 (COLUMN_1 bigint PRIMARY KEY)");
+ assertContainsEventually(() -> execute("SHOW TABLES FROM cassandra.keyspace_1"), resultBuilder(getSession(), createUnboundedVarcharType())
+ .row("table_1")
+ .build(), new Duration(1, MINUTES));
+ assertContains(execute("SHOW COLUMNS FROM cassandra.keyspace_1.table_1"), resultBuilder(getSession(), createUnboundedVarcharType(), createUnboundedVarcharType(), createUnboundedVarcharType(), createUnboundedVarcharType())
+ .row("column_1", "bigint", "", "")
+ .build());
+
+ execute("INSERT INTO keyspace_1.table_1 (column_1) VALUES (1)");
+
+ assertEquals(execute("SELECT column_1 FROM cassandra.keyspace_1.table_1").getRowCount(), 1);
+ assertUpdate("DROP TABLE cassandra.keyspace_1.table_1");
+
+ // when an identifier is unquoted the lowercase and uppercase spelling may be used interchangeable
+ session.execute("DROP KEYSPACE keyspace_1");
+ }
+
+ @Test
+ public void testUppercaseNameEscaped()
+ throws Exception
+ {
+ /*
+ * If an identifier is escaped with double quotes it is stored verbatim
+ *
+ * http://docs.datastax.com/en/cql/3.1/cql/cql_reference/ucase-lcase_r.html
+ */
+ session.execute("CREATE KEYSPACE \"KEYSPACE_2\" WITH REPLICATION = {'class':'SimpleStrategy', 'replication_factor': 1}");
+ assertContainsEventually(() -> execute("SHOW SCHEMAS FROM cassandra"), resultBuilder(getSession(), createUnboundedVarcharType())
+ .row("keyspace_2")
+ .build(), new Duration(1, MINUTES));
+
+ session.execute("CREATE TABLE \"KEYSPACE_2\".\"TABLE_2\" (\"COLUMN_2\" bigint PRIMARY KEY)");
+ assertContainsEventually(() -> execute("SHOW TABLES FROM cassandra.keyspace_2"), resultBuilder(getSession(), createUnboundedVarcharType())
+ .row("table_2")
+ .build(), new Duration(1, MINUTES));
+ assertContains(execute("SHOW COLUMNS FROM cassandra.keyspace_2.table_2"), resultBuilder(getSession(), createUnboundedVarcharType(), createUnboundedVarcharType(), createUnboundedVarcharType(), createUnboundedVarcharType())
+ .row("column_2", "bigint", "", "")
+ .build());
+
+ execute("INSERT INTO \"KEYSPACE_2\".\"TABLE_2\" (\"COLUMN_2\") VALUES (1)");
+
+ assertEquals(execute("SELECT column_2 FROM cassandra.keyspace_2.table_2").getRowCount(), 1);
+ assertUpdate("DROP TABLE cassandra.keyspace_2.table_2");
+
+ // when an identifier is unquoted the lowercase and uppercase spelling may be used interchangeable
+ session.execute("DROP KEYSPACE \"KEYSPACE_2\"");
+ }
+
+ @Test
+ public void testKeyspaceNameAmbiguity()
+ throws Exception
+ {
+ // Identifiers enclosed in double quotes are stored in Cassandra verbatim. It is possible to create 2 keyspaces with names
+ // that have differences only in letters case.
+ session.execute("CREATE KEYSPACE \"KeYsPaCe_3\" WITH REPLICATION = {'class':'SimpleStrategy', 'replication_factor': 1}");
+ session.execute("CREATE KEYSPACE \"kEySpAcE_3\" WITH REPLICATION = {'class':'SimpleStrategy', 'replication_factor': 1}");
+
+ // Although in Presto all the schema and table names are always displayed as lowercase
+ assertContainsEventually(() -> execute("SHOW SCHEMAS FROM cassandra"), resultBuilder(getSession(), createUnboundedVarcharType())
+ .row("keyspace_3")
+ .row("keyspace_3")
+ .build(), new Duration(1, MINUTES));
+
+ // There is no way to figure out what the exactly keyspace we want to retrieve tables from
+ assertQueryFailsEventually(
+ "SHOW TABLES FROM cassandra.keyspace_3",
+ "More than one keyspace has been found for the case insensitive schema name: keyspace_3 -> \\(KeYsPaCe_3, kEySpAcE_3\\)",
+ new Duration(1, MINUTES));
+
+ session.execute("DROP KEYSPACE \"KeYsPaCe_3\"");
+ session.execute("DROP KEYSPACE \"kEySpAcE_3\"");
+ }
+
+ @Test
+ public void testTableNameAmbiguity()
+ throws Exception
+ {
+ session.execute("CREATE KEYSPACE keyspace_4 WITH REPLICATION = {'class':'SimpleStrategy', 'replication_factor': 1}");
+ assertContainsEventually(() -> execute("SHOW SCHEMAS FROM cassandra"), resultBuilder(getSession(), createUnboundedVarcharType())
+ .row("keyspace_4")
+ .build(), new Duration(1, MINUTES));
+
+ // Identifiers enclosed in double quotes are stored in Cassandra verbatim. It is possible to create 2 tables with names
+ // that have differences only in letters case.
+ session.execute("CREATE TABLE keyspace_4.\"TaBlE_4\" (column_4 bigint PRIMARY KEY)");
+ session.execute("CREATE TABLE keyspace_4.\"tAbLe_4\" (column_4 bigint PRIMARY KEY)");
+
+ // Although in Presto all the schema and table names are always displayed as lowercase
+ assertContainsEventually(() -> execute("SHOW TABLES FROM cassandra.keyspace_4"), resultBuilder(getSession(), createUnboundedVarcharType())
+ .row("table_4")
+ .row("table_4")
+ .build(), new Duration(1, MINUTES));
+
+ // There is no way to figure out what the exactly table is being queried
+ assertQueryFailsEventually(
+ "SHOW COLUMNS FROM cassandra.keyspace_4.table_4",
+ "More than one table has been found for the case insensitive table name: table_4 -> \\(TaBlE_4, tAbLe_4\\)",
+ new Duration(1, MINUTES));
+ assertQueryFailsEventually(
+ "SELECT * FROM cassandra.keyspace_4.table_4",
+ "More than one table has been found for the case insensitive table name: table_4 -> \\(TaBlE_4, tAbLe_4\\)",
+ new Duration(1, MINUTES));
+ session.execute("DROP KEYSPACE keyspace_4");
+ }
+
+ @Test
+ public void testColumnNameAmbiguity()
+ throws Exception
+ {
+ session.execute("CREATE KEYSPACE keyspace_5 WITH REPLICATION = {'class':'SimpleStrategy', 'replication_factor': 1}");
+ assertContainsEventually(() -> execute("SHOW SCHEMAS FROM cassandra"), resultBuilder(getSession(), createUnboundedVarcharType())
+ .row("keyspace_5")
+ .build(), new Duration(1, MINUTES));
+
+ session.execute("CREATE TABLE keyspace_5.table_5 (\"CoLuMn_5\" bigint PRIMARY KEY, \"cOlUmN_5\" bigint)");
+ assertContainsEventually(() -> execute("SHOW TABLES FROM cassandra.keyspace_5"), resultBuilder(getSession(), createUnboundedVarcharType())
+ .row("table_5")
+ .build(), new Duration(1, MINUTES));
+
+ assertQueryFailsEventually(
+ "SHOW COLUMNS FROM cassandra.keyspace_5.table_5",
+ "More than one column has been found for the case insensitive column name: column_5 -> \\(CoLuMn_5, cOlUmN_5\\)",
+ new Duration(1, MINUTES));
+ assertQueryFailsEventually(
+ "SELECT * FROM cassandra.keyspace_5.table_5",
+ "More than one column has been found for the case insensitive column name: column_5 -> \\(CoLuMn_5, cOlUmN_5\\)",
+ new Duration(1, MINUTES));
+
+ session.execute("DROP KEYSPACE keyspace_5");
+ }
+
+ @Test
+ public void testInsert()
+ {
+ String sql = "SELECT key, typeuuid, typeinteger, typelong, typebytes, typetimestamp, typeansi, typeboolean, typedecimal, " +
+ "typedouble, typefloat, typeinet, typevarchar, typevarint, typetimeuuid, typelist, typemap, typeset" +
+ " FROM " + TABLE_ALL_TYPES_INSERT;
+ assertEquals(execute(sql).getRowCount(), 0);
+
+ // TODO Following types are not supported now. We need to change null into the value after fixing it
+ // blob, frozen>, inet, list, map, set, timeuuid, decimal, uuid, varint
+ // timestamp can be inserted but the expected and actual values are not same
+ execute("INSERT INTO " + TABLE_ALL_TYPES_INSERT + " (" +
+ "key," +
+ "typeuuid," +
+ "typeinteger," +
+ "typelong," +
+ "typebytes," +
+ "typetimestamp," +
+ "typeansi," +
+ "typeboolean," +
+ "typedecimal," +
+ "typedouble," +
+ "typefloat," +
+ "typeinet," +
+ "typevarchar," +
+ "typevarint," +
+ "typetimeuuid," +
+ "typelist," +
+ "typemap," +
+ "typeset" +
+ ") VALUES (" +
+ "'key1', " +
+ "null, " +
+ "1, " +
+ "1000, " +
+ "null, " +
+ "timestamp '1970-01-01 08:34:05.0', " +
+ "'ansi1', " +
+ "true, " +
+ "null, " +
+ "0.3, " +
+ "cast('0.4' as real), " +
+ "null, " +
+ "'varchar1', " +
+ "null, " +
+ "null, " +
+ "null, " +
+ "null, " +
+ "null " +
+ ")");
+
+ MaterializedResult result = execute(sql);
+ int rowCount = result.getRowCount();
+ assertEquals(rowCount, 1);
+ assertEquals(result.getMaterializedRows().get(0), new MaterializedRow(DEFAULT_PRECISION,
+ "key1",
+ null,
+ 1,
+ 1000L,
+ null,
+ Timestamp.valueOf("1970-01-01 14:04:05.0"),
+ "ansi1",
+ true,
+ null,
+ 0.3,
+ (float) 0.4,
+ null,
+ "varchar1",
+ null,
+ null,
+ null,
+ null,
+ null
+ ));
+
+ // insert null for all datatypes
+ execute("INSERT INTO " + TABLE_ALL_TYPES_INSERT + " (" +
+ "key, typeuuid, typeinteger, typelong, typebytes, typetimestamp, typeansi, typeboolean, typedecimal," +
+ "typedouble, typefloat, typeinet, typevarchar, typevarint, typetimeuuid, typelist, typemap, typeset" +
+ ") VALUES (" +
+ "'key2', null, null, null, null, null, null, null, null," +
+ "null, null, null, null, null, null, null, null, null)");
+ sql = "SELECT key, typeuuid, typeinteger, typelong, typebytes, typetimestamp, typeansi, typeboolean, typedecimal, " +
+ "typedouble, typefloat, typeinet, typevarchar, typevarint, typetimeuuid, typelist, typemap, typeset" +
+ " FROM " + TABLE_ALL_TYPES_INSERT + " WHERE key = 'key2'";
+ result = execute(sql);
+ rowCount = result.getRowCount();
+ assertEquals(rowCount, 1);
+ assertEquals(result.getMaterializedRows().get(0), new MaterializedRow(DEFAULT_PRECISION,
+ "key2", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null));
+
+ // insert into only a subset of columns
+ execute("INSERT INTO " + TABLE_ALL_TYPES_INSERT + " (" +
+ "key, typeinteger, typeansi, typeboolean) VALUES (" +
+ "'key3', 999, 'ansi', false)");
+ sql = "SELECT key, typeuuid, typeinteger, typelong, typebytes, typetimestamp, typeansi, typeboolean, typedecimal, " +
+ "typedouble, typefloat, typeinet, typevarchar, typevarint, typetimeuuid, typelist, typemap, typeset" +
+ " FROM " + TABLE_ALL_TYPES_INSERT + " WHERE key = 'key3'";
+ result = execute(sql);
+ rowCount = result.getRowCount();
+ assertEquals(rowCount, 1);
+ assertEquals(result.getMaterializedRows().get(0), new MaterializedRow(DEFAULT_PRECISION,
+ "key3", null, 999, null, null, null, "ansi", false, null, null, null, null, null, null, null, null, null, null));
}
private void assertSelect(String tableName, boolean createdByPresto)
diff --git a/presto-cassandra/src/test/java/com/facebook/presto/cassandra/TestCassandraTokenSplitManager.java b/presto-cassandra/src/test/java/com/facebook/presto/cassandra/TestCassandraTokenSplitManager.java
index 0a84fd5630864..367ede2cf3900 100644
--- a/presto-cassandra/src/test/java/com/facebook/presto/cassandra/TestCassandraTokenSplitManager.java
+++ b/presto-cassandra/src/test/java/com/facebook/presto/cassandra/TestCassandraTokenSplitManager.java
@@ -14,7 +14,7 @@
package com.facebook.presto.cassandra;
import com.facebook.presto.cassandra.CassandraTokenSplitManager.TokenSplit;
-import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;
import java.util.List;
@@ -27,42 +27,46 @@ public class TestCassandraTokenSplitManager
{
private static final int SPLIT_SIZE = 100;
private static final String KEYSPACE = "test_cassandra_token_split_manager_keyspace";
- private static final String TABLE = "test_cassandra_token_split_manager_table";
private static final int PARTITION_COUNT = 1000;
private CassandraSession session;
private CassandraTokenSplitManager splitManager;
- @BeforeMethod
+ @BeforeClass
public void setUp()
throws Exception
{
EmbeddedCassandra.start();
session = EmbeddedCassandra.getSession();
+ createKeyspace(session, KEYSPACE);
splitManager = new CassandraTokenSplitManager(session, SPLIT_SIZE);
}
@Test
- public void testCassandraTokenSplitManager()
+ public void testEmptyTable()
throws Exception
{
- createKeyspace(session, KEYSPACE);
- session.execute(format("CREATE TABLE %s.%s (key text PRIMARY KEY)", KEYSPACE, TABLE));
-
- EmbeddedCassandra.flush(KEYSPACE, TABLE);
- EmbeddedCassandra.refreshSizeEstimates();
-
- List splits = splitManager.getSplits(KEYSPACE, TABLE);
+ String tableName = "empty_table";
+ session.execute(format("CREATE TABLE %s.%s (key text PRIMARY KEY)", KEYSPACE, tableName));
+ EmbeddedCassandra.refreshSizeEstimates(KEYSPACE, tableName);
+ List splits = splitManager.getSplits(KEYSPACE, tableName);
// even for the empty table at least one split must be produced, in case the statistics are inaccurate
assertEquals(splits.size(), 1);
+ session.execute(format("DROP TABLE %s.%s", KEYSPACE, tableName));
+ }
+ @Test
+ public void testNonEmptyTable()
+ throws Exception
+ {
+ String tableName = "non_empty_table";
+ session.execute(format("CREATE TABLE %s.%s (key text PRIMARY KEY)", KEYSPACE, tableName));
for (int i = 0; i < PARTITION_COUNT; i++) {
- session.execute(format("INSERT INTO %s.%s (key) VALUES ('%s')", KEYSPACE, TABLE, "value" + i));
+ session.execute(format("INSERT INTO %s.%s (key) VALUES ('%s')", KEYSPACE, tableName, "value" + i));
}
- EmbeddedCassandra.flush(KEYSPACE, TABLE);
- EmbeddedCassandra.refreshSizeEstimates();
-
- splits = splitManager.getSplits(KEYSPACE, TABLE);
+ EmbeddedCassandra.refreshSizeEstimates(KEYSPACE, tableName);
+ List splits = splitManager.getSplits(KEYSPACE, tableName);
assertEquals(splits.size(), PARTITION_COUNT / SPLIT_SIZE);
+ session.execute(format("DROP TABLE %s.%s", KEYSPACE, tableName));
}
}
diff --git a/presto-cassandra/src/test/java/com/facebook/presto/cassandra/util/TestCassandraClusteringPredicatesExtractor.java b/presto-cassandra/src/test/java/com/facebook/presto/cassandra/util/TestCassandraClusteringPredicatesExtractor.java
index 5a56a88d3d473..78e5c82610b7d 100644
--- a/presto-cassandra/src/test/java/com/facebook/presto/cassandra/util/TestCassandraClusteringPredicatesExtractor.java
+++ b/presto-cassandra/src/test/java/com/facebook/presto/cassandra/util/TestCassandraClusteringPredicatesExtractor.java
@@ -13,6 +13,7 @@
*/
package com.facebook.presto.cassandra.util;
+import com.datastax.driver.core.VersionNumber;
import com.facebook.presto.cassandra.CassandraClusteringPredicatesExtractor;
import com.facebook.presto.cassandra.CassandraColumnHandle;
import com.facebook.presto.cassandra.CassandraTable;
@@ -26,8 +27,6 @@
import org.testng.annotations.BeforeTest;
import org.testng.annotations.Test;
-import java.util.List;
-
import static com.facebook.presto.spi.type.BigintType.BIGINT;
import static org.testng.Assert.assertEquals;
@@ -38,6 +37,7 @@ public class TestCassandraClusteringPredicatesExtractor
private static CassandraColumnHandle col3;
private static CassandraColumnHandle col4;
private static CassandraTable cassandraTable;
+ private static VersionNumber cassandraVersion;
@BeforeTest
void setUp()
@@ -50,6 +50,8 @@ void setUp()
cassandraTable = new CassandraTable(
new CassandraTableHandle("cassandra", "test", "records"), ImmutableList.of(col1, col2, col3, col4));
+
+ cassandraVersion = VersionNumber.parse("2.1.5");
}
@Test
@@ -60,9 +62,9 @@ public void testBuildClusteringPredicate()
col1, Domain.singleValue(BIGINT, 23L),
col2, Domain.singleValue(BIGINT, 34L),
col4, Domain.singleValue(BIGINT, 26L)));
- CassandraClusteringPredicatesExtractor predicatesExtractor = new CassandraClusteringPredicatesExtractor(cassandraTable.getClusteringKeyColumns(), tupleDomain);
- List predicate = predicatesExtractor.getClusteringKeyPredicates();
- assertEquals(predicate.get(0), new StringBuilder("\"clusteringKey1\" = 34").toString());
+ CassandraClusteringPredicatesExtractor predicatesExtractor = new CassandraClusteringPredicatesExtractor(cassandraTable.getClusteringKeyColumns(), tupleDomain, cassandraVersion);
+ String predicate = predicatesExtractor.getClusteringKeyPredicates();
+ assertEquals(predicate, new StringBuilder("\"clusteringKey1\" = 34").toString());
}
@Test
@@ -72,7 +74,7 @@ public void testGetUnenforcedPredicates()
ImmutableMap.of(
col2, Domain.singleValue(BIGINT, 34L),
col4, Domain.singleValue(BIGINT, 26L)));
- CassandraClusteringPredicatesExtractor predicatesExtractor = new CassandraClusteringPredicatesExtractor(cassandraTable.getClusteringKeyColumns(), tupleDomain);
+ CassandraClusteringPredicatesExtractor predicatesExtractor = new CassandraClusteringPredicatesExtractor(cassandraTable.getClusteringKeyColumns(), tupleDomain, cassandraVersion);
TupleDomain unenforcedPredicates = TupleDomain.withColumnDomains(ImmutableMap.of(col4, Domain.singleValue(BIGINT, 26L)));
assertEquals(predicatesExtractor.getUnenforcedConstraints(), unenforcedPredicates);
}
diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml
index 0bee905c27e36..b890075daed90 100644
--- a/presto-cli/pom.xml
+++ b/presto-cli/pom.xml
@@ -5,7 +5,7 @@
com.facebook.presto
presto-root
- 0.175-SNAPSHOT
+ 0.180-SNAPSHOT
presto-cli
@@ -14,8 +14,6 @@
${project.parent.basedir}
com.facebook.presto.cli.Presto
- false
- ${main-class}
@@ -39,16 +37,6 @@
concurrent
-
- io.airlift
- http-client
-
-
-
- io.airlift
- json
-
-
io.airlift
log
@@ -89,6 +77,11 @@
opencsv
+
+ com.squareup.okhttp3
+ okhttp
+
+
org.testng
diff --git a/presto-cli/src/main/java/com/facebook/presto/cli/ClientOptions.java b/presto-cli/src/main/java/com/facebook/presto/cli/ClientOptions.java
index 819fc938b0b09..12f5b67217df1 100644
--- a/presto-cli/src/main/java/com/facebook/presto/cli/ClientOptions.java
+++ b/presto-cli/src/main/java/com/facebook/presto/cli/ClientOptions.java
@@ -18,10 +18,8 @@
import com.google.common.collect.ImmutableMap;
import com.google.common.net.HostAndPort;
import io.airlift.airline.Option;
-import io.airlift.http.client.spnego.KerberosConfig;
import io.airlift.units.Duration;
-import java.io.File;
import java.net.URI;
import java.net.URISyntaxException;
import java.nio.charset.CharsetEncoder;
@@ -33,6 +31,7 @@
import java.util.Optional;
import java.util.TimeZone;
+import static com.facebook.presto.client.KerberosUtil.defaultCredentialCachePath;
import static com.google.common.base.Preconditions.checkArgument;
import static java.nio.charset.StandardCharsets.US_ASCII;
import static java.util.Collections.emptyMap;
@@ -58,7 +57,7 @@ public class ClientOptions
public String krb5KeytabPath = "/etc/krb5.keytab";
@Option(name = "--krb5-credential-cache-path", title = "krb5 credential cache path", description = "Kerberos credential cache path")
- public String krb5CredentialCachePath = defaultCredentialCachePath();
+ public String krb5CredentialCachePath = defaultCredentialCachePath().orElse(null);
@Option(name = "--krb5-principal", title = "krb5 principal", description = "Kerberos principal to be used")
public String krb5Principal;
@@ -114,6 +113,9 @@ public class ClientOptions
@Option(name = "--socks-proxy", title = "socks-proxy", description = "SOCKS proxy to use for server connections")
public HostAndPort socksProxy;
+ @Option(name = "--http-proxy", title = "http-proxy", description = "HTTP proxy to use for server connections")
+ public HostAndPort httpProxy;
+
@Option(name = "--client-request-timeout", title = "client request timeout", description = "Client request timeout (default: 2m)")
public Duration clientRequestTimeout = new Duration(2, MINUTES);
@@ -146,22 +148,6 @@ public ClientSession toClientSession()
clientRequestTimeout);
}
- public KerberosConfig toKerberosConfig()
- {
- KerberosConfig config = new KerberosConfig();
- if (krb5ConfigPath != null) {
- config.setConfig(new File(krb5ConfigPath));
- }
- if (krb5KeytabPath != null) {
- config.setKeytab(new File(krb5KeytabPath));
- }
- if (krb5CredentialCachePath != null) {
- config.setCredentialCache(new File(krb5CredentialCachePath));
- }
- config.setUseCanonicalHostname(!krb5DisableRemoteServiceHostnameCanonicalization);
- return config;
- }
-
public static URI parseServer(String server)
{
server = server.toLowerCase(ENGLISH);
@@ -171,7 +157,7 @@ public static URI parseServer(String server)
HostAndPort host = HostAndPort.fromString(server);
try {
- return new URI("http", null, host.getHostText(), host.getPortOrDefault(80), null, null, null);
+ return new URI("http", null, host.getHost(), host.getPortOrDefault(80), null, null, null);
}
catch (URISyntaxException e) {
throw new IllegalArgumentException(e);
@@ -191,15 +177,6 @@ public static Map toProperties(List sessi
return builder.build();
}
- private static String defaultCredentialCachePath()
- {
- String value = System.getenv("KRB5CCNAME");
- if (value != null && value.startsWith("FILE:")) {
- return value.substring("FILE:".length());
- }
- return value;
- }
-
public static final class ClientSessionProperty
{
private static final Splitter NAME_VALUE_SPLITTER = Splitter.on('=').limit(2);
diff --git a/presto-cli/src/main/java/com/facebook/presto/cli/Console.java b/presto-cli/src/main/java/com/facebook/presto/cli/Console.java
index 982ea5ed4c7c0..ef72bdad3d7ac 100644
--- a/presto-cli/src/main/java/com/facebook/presto/cli/Console.java
+++ b/presto-cli/src/main/java/com/facebook/presto/cli/Console.java
@@ -27,7 +27,6 @@
import com.google.common.io.Files;
import io.airlift.airline.Command;
import io.airlift.airline.HelpOption;
-import io.airlift.http.client.spnego.KerberosConfig;
import io.airlift.log.Logging;
import io.airlift.log.LoggingConfiguration;
import io.airlift.units.Duration;
@@ -51,6 +50,7 @@
import static com.facebook.presto.cli.Completion.commandCompleter;
import static com.facebook.presto.cli.Completion.lowerCaseCommandCompleter;
import static com.facebook.presto.cli.Help.getHelpText;
+import static com.facebook.presto.cli.QueryPreprocessor.preprocessQuery;
import static com.facebook.presto.client.ClientSession.stripTransactionId;
import static com.facebook.presto.client.ClientSession.withCatalogAndSchema;
import static com.facebook.presto.client.ClientSession.withPreparedStatements;
@@ -93,7 +93,6 @@ public class Console
public void run()
{
ClientSession session = clientOptions.toClientSession();
- KerberosConfig kerberosConfig = clientOptions.toKerberosConfig();
boolean hasQuery = !Strings.isNullOrEmpty(clientOptions.execute);
boolean isFromFile = !Strings.isNullOrEmpty(clientOptions.file);
@@ -124,9 +123,10 @@ public void run()
AtomicBoolean exiting = new AtomicBoolean();
interruptThreadOnExit(Thread.currentThread(), exiting);
- try (QueryRunner queryRunner = QueryRunner.create(
+ try (QueryRunner queryRunner = new QueryRunner(
session,
Optional.ofNullable(clientOptions.socksProxy),
+ Optional.ofNullable(clientOptions.httpProxy),
Optional.ofNullable(clientOptions.keystorePath),
Optional.ofNullable(clientOptions.keystorePassword),
Optional.ofNullable(clientOptions.truststorePath),
@@ -135,8 +135,11 @@ public void run()
clientOptions.password ? Optional.of(getPassword()) : Optional.empty(),
Optional.ofNullable(clientOptions.krb5Principal),
Optional.ofNullable(clientOptions.krb5RemoteServiceName),
- clientOptions.authenticationEnabled,
- kerberosConfig)) {
+ Optional.ofNullable(clientOptions.krb5ConfigPath),
+ Optional.ofNullable(clientOptions.krb5KeytabPath),
+ Optional.ofNullable(clientOptions.krb5CredentialCachePath),
+ !clientOptions.krb5DisableRemoteServiceHostnameCanonicalization,
+ clientOptions.authenticationEnabled)) {
if (hasQuery) {
executeCommand(queryRunner, query, clientOptions.outputFormat);
}
@@ -314,7 +317,22 @@ private static void executeCommand(QueryRunner queryRunner, String query, Output
private static void process(QueryRunner queryRunner, String sql, OutputFormat outputFormat, boolean interactive)
{
- try (Query query = queryRunner.startQuery(sql)) {
+ String finalSql;
+ try {
+ finalSql = preprocessQuery(
+ Optional.ofNullable(queryRunner.getSession().getCatalog()),
+ Optional.ofNullable(queryRunner.getSession().getSchema()),
+ sql);
+ }
+ catch (QueryPreprocessorException e) {
+ System.err.println(e.getMessage());
+ if (queryRunner.getSession().isDebug()) {
+ e.printStackTrace();
+ }
+ return;
+ }
+
+ try (Query query = queryRunner.startQuery(finalSql)) {
query.renderOutput(System.out, outputFormat, interactive);
ClientSession session = queryRunner.getSession();
diff --git a/presto-cli/src/main/java/com/facebook/presto/cli/LdapRequestFilter.java b/presto-cli/src/main/java/com/facebook/presto/cli/LdapRequestFilter.java
deleted file mode 100644
index de40f0e32cc88..0000000000000
--- a/presto-cli/src/main/java/com/facebook/presto/cli/LdapRequestFilter.java
+++ /dev/null
@@ -1,48 +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.cli;
-
-import com.google.common.net.HttpHeaders;
-import io.airlift.http.client.HttpRequestFilter;
-import io.airlift.http.client.Request;
-
-import java.util.Base64;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static io.airlift.http.client.Request.Builder.fromRequest;
-import static java.nio.charset.StandardCharsets.ISO_8859_1;
-import static java.util.Objects.requireNonNull;
-
-public class LdapRequestFilter
- implements HttpRequestFilter
-{
- private final String user;
- private final String password;
-
- public LdapRequestFilter(String user, String password)
- {
- this.user = requireNonNull(user, "user is null");
- checkArgument(!user.contains(":"), "Illegal character ':' found in username");
- this.password = requireNonNull(password, "password is null");
- }
-
- @Override
- public Request filterRequest(Request request)
- {
- String value = "Basic " + Base64.getEncoder().encodeToString((user + ":" + password).getBytes(ISO_8859_1));
- return fromRequest(request)
- .addHeader(HttpHeaders.AUTHORIZATION, value)
- .build();
- }
-}
diff --git a/presto-cli/src/main/java/com/facebook/presto/cli/QueryPreprocessor.java b/presto-cli/src/main/java/com/facebook/presto/cli/QueryPreprocessor.java
new file mode 100644
index 0000000000000..2ce2a99a16180
--- /dev/null
+++ b/presto-cli/src/main/java/com/facebook/presto/cli/QueryPreprocessor.java
@@ -0,0 +1,213 @@
+/*
+ * 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.cli;
+
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableList;
+import com.google.common.io.CharStreams;
+import io.airlift.units.Duration;
+import sun.misc.Signal;
+import sun.misc.SignalHandler;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static com.facebook.presto.cli.ConsolePrinter.REAL_TERMINAL;
+import static com.google.common.base.Strings.emptyToNull;
+import static com.google.common.base.Strings.nullToEmpty;
+import static com.google.common.base.Throwables.propagateIfPossible;
+import static io.airlift.concurrent.MoreFutures.tryGetFutureValue;
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+
+public final class QueryPreprocessor
+{
+ public static final String ENV_PREPROCESSOR = "PRESTO_PREPROCESSOR";
+ public static final String ENV_PREPROCESSOR_TIMEOUT = "PRESTO_PREPROCESSOR_TIMEOUT";
+ public static final String ENV_PRESTO_CATALOG = "PRESTO_CATALOG";
+ public static final String ENV_PRESTO_SCHEMA = "PRESTO_SCHEMA";
+ private static final Duration DEFAULT_PREPROCESSOR_TIMEOUT = new Duration(10, SECONDS);
+
+ private static final Signal SIGINT = new Signal("INT");
+ private static final String PREPROCESSING_QUERY_MESSAGE = "Preprocessing query...";
+
+ private QueryPreprocessor() {}
+
+ public static String preprocessQuery(Optional catalog, Optional schema, String query)
+ throws QueryPreprocessorException
+ {
+ Duration timeout = DEFAULT_PREPROCESSOR_TIMEOUT;
+ String timeoutEnvironment = nullToEmpty(System.getenv(ENV_PREPROCESSOR_TIMEOUT)).trim();
+ if (!timeoutEnvironment.isEmpty()) {
+ timeout = Duration.valueOf(timeoutEnvironment);
+ }
+
+ String preprocessorCommand = System.getenv(ENV_PREPROCESSOR);
+ if (emptyToNull(preprocessorCommand) == null) {
+ return query;
+ }
+ return preprocessQuery(catalog, schema, query, ImmutableList.of("/bin/sh", "-c", preprocessorCommand), timeout);
+ }
+
+ public static String preprocessQuery(Optional catalog, Optional schema, String query, List preprocessorCommand, Duration timeout)
+ throws QueryPreprocessorException
+ {
+ Thread clientThread = Thread.currentThread();
+ SignalHandler oldHandler = Signal.handle(SIGINT, signal -> clientThread.interrupt());
+ try {
+ if (REAL_TERMINAL) {
+ System.out.print(PREPROCESSING_QUERY_MESSAGE);
+ System.out.flush();
+ }
+ return preprocessQueryInternal(catalog, schema, query, preprocessorCommand, timeout);
+ }
+ finally {
+ if (REAL_TERMINAL) {
+ System.out.print("\r" + Strings.repeat(" ", PREPROCESSING_QUERY_MESSAGE.length()) + "\r");
+ System.out.flush();
+ }
+ Signal.handle(SIGINT, oldHandler);
+ Thread.interrupted(); // clear interrupt status
+ }
+ }
+
+ private static String preprocessQueryInternal(Optional catalog, Optional schema, String query, List preprocessorCommand, Duration timeout)
+ throws QueryPreprocessorException
+ {
+ // execute the process in a child thread so we can better handle interruption and timeouts
+ AtomicReference processReference = new AtomicReference<>();
+
+ Future task = executeInNewThread("Query preprocessor", () -> {
+ String result;
+ int exitCode;
+ Future readStderr;
+ try {
+ ProcessBuilder processBuilder = new ProcessBuilder(preprocessorCommand);
+ processBuilder.environment().put(ENV_PRESTO_CATALOG, catalog.orElse(""));
+ processBuilder.environment().put(ENV_PRESTO_SCHEMA, schema.orElse(""));
+
+ Process process = processBuilder.start();
+ processReference.set(process);
+
+ Future> writeOutput = null;
+ try {
+ // write query to process standard out
+ writeOutput = executeInNewThread("Query preprocessor output", () -> {
+ try (OutputStream outputStream = process.getOutputStream()) {
+ outputStream.write(query.getBytes(UTF_8));
+ }
+ return null;
+ });
+
+ // read stderr
+ readStderr = executeInNewThread("Query preprocessor read stderr", () -> {
+ StringBuilder builder = new StringBuilder();
+ try (InputStream inputStream = process.getErrorStream()) {
+ CharStreams.copy(new InputStreamReader(inputStream, UTF_8), builder);
+ }
+ catch (IOException | RuntimeException ignored) {
+ }
+ return builder.toString();
+ });
+
+ // read response
+ try (InputStream inputStream = process.getInputStream()) {
+ result = CharStreams.toString(new InputStreamReader(inputStream, UTF_8));
+ }
+
+ // verify output was written successfully
+ try {
+ writeOutput.get();
+ }
+ catch (ExecutionException e) {
+ throw e.getCause();
+ }
+
+ // wait for process to finish
+ exitCode = process.waitFor();
+ }
+ finally {
+ process.destroyForcibly();
+ if (writeOutput != null) {
+ writeOutput.cancel(true);
+ }
+ }
+ }
+ catch (QueryPreprocessorException e) {
+ throw e;
+ }
+ catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new QueryPreprocessorException("Interrupted while preprocessing query");
+ }
+ catch (Throwable e) {
+ throw new QueryPreprocessorException("Error preprocessing query: " + e.getMessage(), e);
+ }
+
+ // check we got a valid exit code
+ if (exitCode != 0) {
+ Optional errorMessage = tryGetFutureValue(readStderr, 100, MILLISECONDS)
+ .flatMap(value -> Optional.ofNullable(emptyToNull(value.trim())));
+
+ throw new QueryPreprocessorException("Query preprocessor exited " + exitCode +
+ errorMessage.map(message1 -> "\n===\n" + message1 + "\n===").orElse(""));
+ }
+ return result;
+ });
+
+ try {
+ return task.get(timeout.toMillis(), MILLISECONDS);
+ }
+ catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new QueryPreprocessorException("Interrupted while preprocessing query");
+ }
+ catch (ExecutionException e) {
+ Throwable cause = e.getCause();
+ propagateIfPossible(cause, QueryPreprocessorException.class);
+ throw new QueryPreprocessorException("Error preprocessing query: " + cause.getMessage(), cause);
+ }
+ catch (TimeoutException e) {
+ throw new QueryPreprocessorException("Timed out waiting for query preprocessor after " + timeout);
+ }
+ finally {
+ Process process = processReference.get();
+ if (process != null) {
+ process.destroyForcibly();
+ }
+ task.cancel(true);
+ }
+ }
+
+ private static Future executeInNewThread(String threadName, Callable callable)
+ {
+ FutureTask task = new FutureTask<>(callable);
+ Thread thread = new Thread(task);
+ thread.setName(threadName);
+ thread.setDaemon(true);
+ thread.start();
+ return task;
+ }
+}
diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestingTableHandle.java b/presto-cli/src/main/java/com/facebook/presto/cli/QueryPreprocessorException.java
similarity index 59%
rename from presto-main/src/test/java/com/facebook/presto/sql/planner/TestingTableHandle.java
rename to presto-cli/src/main/java/com/facebook/presto/cli/QueryPreprocessorException.java
index 1db8042dfa504..62f4505f916f7 100644
--- a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestingTableHandle.java
+++ b/presto-cli/src/main/java/com/facebook/presto/cli/QueryPreprocessorException.java
@@ -11,19 +11,18 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package com.facebook.presto.sql.planner;
+package com.facebook.presto.cli;
-import com.facebook.presto.spi.ConnectorTableHandle;
-import com.fasterxml.jackson.annotation.JsonValue;
-
-public class TestingTableHandle
- implements ConnectorTableHandle
+public class QueryPreprocessorException
+ extends Exception
{
- // Jackson refuses to serialize this class otherwise because it's empty.
- @JsonValue
- @Override
- public String toString()
+ public QueryPreprocessorException(String message)
+ {
+ super(message);
+ }
+
+ public QueryPreprocessorException(String message, Throwable cause)
{
- return getClass().getSimpleName();
+ super(message, cause);
}
}
diff --git a/presto-cli/src/main/java/com/facebook/presto/cli/QueryRunner.java b/presto-cli/src/main/java/com/facebook/presto/cli/QueryRunner.java
index 80fa038eafabc..83031ba99574b 100644
--- a/presto-cli/src/main/java/com/facebook/presto/cli/QueryRunner.java
+++ b/presto-cli/src/main/java/com/facebook/presto/cli/QueryRunner.java
@@ -13,39 +13,37 @@
*/
package com.facebook.presto.cli;
+import com.facebook.presto.client.ClientException;
import com.facebook.presto.client.ClientSession;
-import com.facebook.presto.client.QueryResults;
import com.facebook.presto.client.StatementClient;
-import com.google.common.collect.ImmutableList;
import com.google.common.net.HostAndPort;
-import io.airlift.http.client.HttpClient;
-import io.airlift.http.client.HttpClientConfig;
-import io.airlift.http.client.HttpRequestFilter;
-import io.airlift.http.client.jetty.JettyHttpClient;
-import io.airlift.http.client.spnego.KerberosConfig;
-import io.airlift.json.JsonCodec;
-import io.airlift.units.Duration;
+import okhttp3.OkHttpClient;
import java.io.Closeable;
+import java.io.File;
import java.util.Optional;
-import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
+import static com.facebook.presto.client.OkHttpUtil.basicAuth;
+import static com.facebook.presto.client.OkHttpUtil.setupHttpProxy;
+import static com.facebook.presto.client.OkHttpUtil.setupKerberos;
+import static com.facebook.presto.client.OkHttpUtil.setupSocksProxy;
+import static com.facebook.presto.client.OkHttpUtil.setupSsl;
+import static com.facebook.presto.client.OkHttpUtil.setupTimeouts;
import static com.google.common.base.Preconditions.checkArgument;
-import static io.airlift.json.JsonCodec.jsonCodec;
import static java.util.Objects.requireNonNull;
+import static java.util.concurrent.TimeUnit.SECONDS;
public class QueryRunner
implements Closeable
{
- private final JsonCodec queryResultsCodec;
private final AtomicReference session;
- private final HttpClient httpClient;
+ private final OkHttpClient httpClient;
public QueryRunner(
ClientSession session,
- JsonCodec queryResultsCodec,
Optional socksProxy,
+ Optional httpProxy,
Optional keystorePath,
Optional keystorePassword,
Optional truststorePath,
@@ -54,24 +52,34 @@ public QueryRunner(
Optional password,
Optional kerberosPrincipal,
Optional kerberosRemoteServiceName,
- boolean authenticationEnabled,
- KerberosConfig kerberosConfig)
+ Optional kerberosConfigPath,
+ Optional kerberosKeytabPath,
+ Optional kerberosCredentialCachePath,
+ boolean kerberosUseCanonicalHostname,
+ boolean kerberosEnabled)
{
this.session = new AtomicReference<>(requireNonNull(session, "session is null"));
- this.queryResultsCodec = requireNonNull(queryResultsCodec, "queryResultsCodec is null");
- this.httpClient = new JettyHttpClient(
- getHttpClientConfig(
- socksProxy,
- keystorePath,
- keystorePassword,
- truststorePath,
- truststorePassword,
- kerberosPrincipal,
- kerberosRemoteServiceName,
- authenticationEnabled),
- kerberosConfig,
- Optional.empty(),
- getRequestFilters(session, user, password));
+
+ OkHttpClient.Builder builder = new OkHttpClient.Builder();
+
+ setupTimeouts(builder, 5, SECONDS);
+ setupSocksProxy(builder, socksProxy);
+ setupHttpProxy(builder, httpProxy);
+ setupSsl(builder, keystorePath, keystorePassword, truststorePath, truststorePassword);
+ setupBasicAuth(builder, session, user, password);
+
+ if (kerberosEnabled) {
+ setupKerberos(
+ builder,
+ kerberosRemoteServiceName.orElseThrow(() -> new ClientException("Kerberos remote service name must be set")),
+ kerberosUseCanonicalHostname,
+ kerberosPrincipal,
+ kerberosConfigPath.map(File::new),
+ kerberosKeytabPath.map(File::new),
+ kerberosCredentialCachePath.map(File::new));
+ }
+
+ this.httpClient = builder.build();
}
public ClientSession getSession()
@@ -91,79 +99,26 @@ public Query startQuery(String query)
public StatementClient startInternalQuery(String query)
{
- return new StatementClient(httpClient, queryResultsCodec, session.get(), query);
+ return new StatementClient(httpClient, session.get(), query);
}
@Override
public void close()
{
- httpClient.close();
+ httpClient.dispatcher().executorService().shutdown();
+ httpClient.connectionPool().evictAll();
}
- public static QueryRunner create(
+ private static void setupBasicAuth(
+ OkHttpClient.Builder clientBuilder,
ClientSession session,
- Optional socksProxy,
- Optional keystorePath,
- Optional keystorePassword,
- Optional truststorePath,
- Optional truststorePassword,
Optional user,
- Optional password,
- Optional kerberosPrincipal,
- Optional kerberosRemoteServiceName,
- boolean authenticationEnabled,
- KerberosConfig kerberosConfig)
- {
- return new QueryRunner(
- session,
- jsonCodec(QueryResults.class),
- socksProxy,
- keystorePath,
- keystorePassword,
- truststorePath,
- truststorePassword,
- user,
- password,
- kerberosPrincipal,
- kerberosRemoteServiceName,
- authenticationEnabled,
- kerberosConfig);
- }
-
- private static HttpClientConfig getHttpClientConfig(
- Optional socksProxy,
- Optional keystorePath,
- Optional keystorePassword,
- Optional truststorePath,
- Optional truststorePassword,
- Optional kerberosPrincipal,
- Optional kerberosRemoteServiceName,
- boolean authenticationEnabled)
- {
- HttpClientConfig httpClientConfig = new HttpClientConfig()
- .setConnectTimeout(new Duration(5, TimeUnit.SECONDS))
- .setRequestTimeout(new Duration(5, TimeUnit.SECONDS));
-
- socksProxy.ifPresent(httpClientConfig::setSocksProxy);
-
- httpClientConfig.setAuthenticationEnabled(authenticationEnabled);
-
- keystorePath.ifPresent(httpClientConfig::setKeyStorePath);
- keystorePassword.ifPresent(httpClientConfig::setKeyStorePassword);
- truststorePath.ifPresent(httpClientConfig::setTrustStorePath);
- truststorePassword.ifPresent(httpClientConfig::setTrustStorePassword);
- kerberosPrincipal.ifPresent(httpClientConfig::setKerberosPrincipal);
- kerberosRemoteServiceName.ifPresent(httpClientConfig::setKerberosRemoteServiceName);
-
- return httpClientConfig;
- }
-
- private static Iterable getRequestFilters(ClientSession session, Optional user, Optional password)
+ Optional password)
{
if (user.isPresent() && password.isPresent()) {
- checkArgument(session.getServer().getScheme().equalsIgnoreCase("https"), "Authentication using username/password requires HTTPS to be enabled");
- return ImmutableList.of(new LdapRequestFilter(user.get(), password.get()));
+ checkArgument(session.getServer().getScheme().equalsIgnoreCase("https"),
+ "Authentication using username/password requires HTTPS to be enabled");
+ clientBuilder.addInterceptor(basicAuth(user.get(), password.get()));
}
- return ImmutableList.of();
}
}
diff --git a/presto-cli/src/main/java/com/facebook/presto/cli/StatusPrinter.java b/presto-cli/src/main/java/com/facebook/presto/cli/StatusPrinter.java
index 7d45a47cb7ae3..80f65a834ec2d 100644
--- a/presto-cli/src/main/java/com/facebook/presto/cli/StatusPrinter.java
+++ b/presto-cli/src/main/java/com/facebook/presto/cli/StatusPrinter.java
@@ -99,7 +99,7 @@ public void printInitialStatusUpdates()
// check for keyboard input
int key = readKey();
if (key == CTRL_P) {
- partialCancel();
+ client.cancelLeafStage();
}
else if (key == CTRL_C) {
updateScreen();
@@ -406,16 +406,6 @@ private void printStageTree(StageStats stage, String indent, AtomicInteger stage
}
}
- private void partialCancel()
- {
- try {
- client.cancelLeafStage(new Duration(1, SECONDS));
- }
- catch (RuntimeException e) {
- log.debug(e, "error canceling leaf stage");
- }
- }
-
private void reprintLine(String line)
{
console.reprintLine(line);
diff --git a/presto-cli/src/test/java/com/facebook/presto/cli/TestTableNameCompleter.java b/presto-cli/src/test/java/com/facebook/presto/cli/TestTableNameCompleter.java
index 3c1bf78c27cda..e9f4275cfd64c 100644
--- a/presto-cli/src/test/java/com/facebook/presto/cli/TestTableNameCompleter.java
+++ b/presto-cli/src/test/java/com/facebook/presto/cli/TestTableNameCompleter.java
@@ -27,7 +27,12 @@ public class TestTableNameCompleter
public void testAutoCompleteWithoutSchema()
{
ClientSession session = new ClientOptions().toClientSession();
- QueryRunner runner = QueryRunner.create(session,
+ QueryRunner runner = new QueryRunner(
+ session,
+ Optional.empty(),
+ Optional.empty(),
+ Optional.empty(),
+ Optional.empty(),
Optional.empty(),
Optional.empty(),
Optional.empty(),
@@ -38,7 +43,7 @@ public void testAutoCompleteWithoutSchema()
Optional.empty(),
Optional.empty(),
false,
- null);
+ false);
TableNameCompleter completer = new TableNameCompleter(runner);
assertEquals(completer.complete("SELECT is_infi", 14, ImmutableList.of()), 7);
}
diff --git a/presto-client/pom.xml b/presto-client/pom.xml
index 0dae37aa67d4a..f0e1b7c6cdcfc 100644
--- a/presto-client/pom.xml
+++ b/presto-client/pom.xml
@@ -5,7 +5,7 @@
com.facebook.presto
presto-root
- 0.175-SNAPSHOT
+ 0.180-SNAPSHOT
presto-client
@@ -46,11 +46,6 @@
jackson-databind
-
- io.airlift
- http-client
-
-
io.airlift
json
@@ -66,6 +61,11 @@
guava
+
+ com.squareup.okhttp3
+ okhttp
+
+
org.testng
diff --git a/presto-client/src/main/java/com/facebook/presto/client/ClientException.java b/presto-client/src/main/java/com/facebook/presto/client/ClientException.java
new file mode 100644
index 0000000000000..e9cc75ce70f48
--- /dev/null
+++ b/presto-client/src/main/java/com/facebook/presto/client/ClientException.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.client;
+
+public class ClientException
+ extends RuntimeException
+{
+ public ClientException(String message)
+ {
+ super(message);
+ }
+
+ public ClientException(String message, Throwable cause)
+ {
+ super(message, cause);
+ }
+}
diff --git a/presto-client/src/main/java/com/facebook/presto/client/JsonResponse.java b/presto-client/src/main/java/com/facebook/presto/client/JsonResponse.java
new file mode 100644
index 0000000000000..12609e66296a3
--- /dev/null
+++ b/presto-client/src/main/java/com/facebook/presto/client/JsonResponse.java
@@ -0,0 +1,156 @@
+/*
+ * 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.client;
+
+import io.airlift.json.JsonCodec;
+import okhttp3.Headers;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+
+import static com.google.common.base.MoreObjects.toStringHelper;
+import static com.google.common.net.HttpHeaders.LOCATION;
+import static java.lang.String.format;
+import static java.util.Objects.requireNonNull;
+
+public final class JsonResponse
+{
+ private final int statusCode;
+ private final String statusMessage;
+ private final Headers headers;
+ private final String responseBody;
+ private final boolean hasValue;
+ private final T value;
+ private final IllegalArgumentException exception;
+
+ private JsonResponse(int statusCode, String statusMessage, Headers headers, String responseBody)
+ {
+ this.statusCode = statusCode;
+ this.statusMessage = statusMessage;
+ this.headers = requireNonNull(headers, "headers is null");
+ this.responseBody = requireNonNull(responseBody, "responseBody is null");
+
+ this.hasValue = false;
+ this.value = null;
+ this.exception = null;
+ }
+
+ private JsonResponse(int statusCode, String statusMessage, Headers headers, String responseBody, JsonCodec jsonCodec)
+ {
+ this.statusCode = statusCode;
+ this.statusMessage = statusMessage;
+ this.headers = requireNonNull(headers, "headers is null");
+ this.responseBody = requireNonNull(responseBody, "responseBody is null");
+
+ T value = null;
+ IllegalArgumentException exception = null;
+ try {
+ value = jsonCodec.fromJson(responseBody);
+ }
+ catch (IllegalArgumentException e) {
+ exception = new IllegalArgumentException(format("Unable to create %s from JSON response:\n[%s]", jsonCodec.getType(), responseBody), e);
+ }
+ this.hasValue = (exception == null);
+ this.value = value;
+ this.exception = exception;
+ }
+
+ public int getStatusCode()
+ {
+ return statusCode;
+ }
+
+ public String getStatusMessage()
+ {
+ return statusMessage;
+ }
+
+ public Headers getHeaders()
+ {
+ return headers;
+ }
+
+ public boolean hasValue()
+ {
+ return hasValue;
+ }
+
+ public T getValue()
+ {
+ if (!hasValue) {
+ throw new IllegalStateException("Response does not contain a JSON value", exception);
+ }
+ return value;
+ }
+
+ public String getResponseBody()
+ {
+ return responseBody;
+ }
+
+ @Nullable
+ public IllegalArgumentException getException()
+ {
+ return exception;
+ }
+
+ @Override
+ public String toString()
+ {
+ return toStringHelper(this)
+ .add("statusCode", statusCode)
+ .add("statusMessage", statusMessage)
+ .add("headers", headers.toMultimap())
+ .add("hasValue", hasValue)
+ .add("value", value)
+ .omitNullValues()
+ .toString();
+ }
+
+ public static JsonResponse execute(JsonCodec codec, OkHttpClient client, Request request)
+ {
+ try (Response response = client.newCall(request).execute()) {
+ // TODO: fix in OkHttp: https://github.com/square/okhttp/issues/3111
+ if ((response.code() == 307) || (response.code() == 308)) {
+ String location = response.header(LOCATION);
+ if (location != null) {
+ request = request.newBuilder().url(location).build();
+ return execute(codec, client, request);
+ }
+ }
+
+ ResponseBody responseBody = requireNonNull(response.body());
+ String body = responseBody.string();
+ if (isJson(responseBody.contentType())) {
+ return new JsonResponse<>(response.code(), response.message(), response.headers(), body, codec);
+ }
+ return new JsonResponse<>(response.code(), response.message(), response.headers(), body);
+ }
+ catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+ }
+
+ private static boolean isJson(MediaType type)
+ {
+ return (type != null) && "application".equals(type.type()) && "json".equals(type.subtype());
+ }
+}
diff --git a/presto-client/src/main/java/com/facebook/presto/client/KerberosUtil.java b/presto-client/src/main/java/com/facebook/presto/client/KerberosUtil.java
new file mode 100644
index 0000000000000..914def58b2024
--- /dev/null
+++ b/presto-client/src/main/java/com/facebook/presto/client/KerberosUtil.java
@@ -0,0 +1,35 @@
+/*
+ * 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.client;
+
+import java.util.Optional;
+
+import static com.google.common.base.Strings.emptyToNull;
+import static com.google.common.base.Strings.nullToEmpty;
+
+public final class KerberosUtil
+{
+ private static final String FILE_PREFIX = "FILE:";
+
+ private KerberosUtil() {}
+
+ public static Optional defaultCredentialCachePath()
+ {
+ String value = nullToEmpty(System.getenv("KRB5CCNAME"));
+ if (value.startsWith(FILE_PREFIX)) {
+ value = value.substring(FILE_PREFIX.length());
+ }
+ return Optional.ofNullable(emptyToNull(value));
+ }
+}
diff --git a/presto-client/src/main/java/com/facebook/presto/client/OkHttpUtil.java b/presto-client/src/main/java/com/facebook/presto/client/OkHttpUtil.java
new file mode 100644
index 0000000000000..4b788a56535ca
--- /dev/null
+++ b/presto-client/src/main/java/com/facebook/presto/client/OkHttpUtil.java
@@ -0,0 +1,192 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.facebook.presto.client;
+
+import com.google.common.net.HostAndPort;
+import okhttp3.Call;
+import okhttp3.Callback;
+import okhttp3.Credentials;
+import okhttp3.Interceptor;
+import okhttp3.OkHttpClient;
+import okhttp3.Response;
+
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.InetSocketAddress;
+import java.net.Proxy;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+
+import static com.google.common.net.HttpHeaders.AUTHORIZATION;
+import static com.google.common.net.HttpHeaders.USER_AGENT;
+import static java.net.Proxy.Type.HTTP;
+import static java.net.Proxy.Type.SOCKS;
+import static java.util.Objects.requireNonNull;
+
+public final class OkHttpUtil
+{
+ private OkHttpUtil() {}
+
+ public static class NullCallback
+ implements Callback
+ {
+ @Override
+ public void onFailure(Call call, IOException e) {}
+
+ @Override
+ public void onResponse(Call call, Response response) {}
+ }
+
+ public static Interceptor userAgent(String userAgent)
+ {
+ return chain -> chain.proceed(chain.request().newBuilder()
+ .header(USER_AGENT, userAgent)
+ .build());
+ }
+
+ public static Interceptor basicAuth(String user, String password)
+ {
+ requireNonNull(user, "user is null");
+ requireNonNull(password, "password is null");
+ if (user.contains(":")) {
+ throw new ClientException("Illegal character ':' found in username");
+ }
+
+ String credential = Credentials.basic(user, password);
+ return chain -> chain.proceed(chain.request().newBuilder()
+ .header(AUTHORIZATION, credential)
+ .build());
+ }
+
+ public static void setupTimeouts(OkHttpClient.Builder clientBuilder, int timeout, TimeUnit unit)
+ {
+ clientBuilder
+ .connectTimeout(timeout, unit)
+ .readTimeout(timeout, unit)
+ .writeTimeout(timeout, unit);
+ }
+
+ public static void setupSocksProxy(OkHttpClient.Builder clientBuilder, Optional socksProxy)
+ {
+ setupProxy(clientBuilder, socksProxy, SOCKS);
+ }
+
+ public static void setupHttpProxy(OkHttpClient.Builder clientBuilder, Optional httpProxy)
+ {
+ setupProxy(clientBuilder, httpProxy, HTTP);
+ }
+
+ public static void setupProxy(OkHttpClient.Builder clientBuilder, Optional proxy, Proxy.Type type)
+ {
+ proxy.map(OkHttpUtil::toUnresolvedAddress)
+ .map(address -> new Proxy(type, address))
+ .ifPresent(clientBuilder::proxy);
+ }
+
+ private static InetSocketAddress toUnresolvedAddress(HostAndPort address)
+ {
+ return InetSocketAddress.createUnresolved(address.getHost(), address.getPort());
+ }
+
+ public static void setupSsl(
+ OkHttpClient.Builder clientBuilder,
+ Optional keyStorePath,
+ Optional keyStorePassword,
+ Optional trustStorePath,
+ Optional trustStorePassword)
+ {
+ if (!keyStorePath.isPresent() && !trustStorePath.isPresent()) {
+ return;
+ }
+
+ try {
+ // load KeyStore if configured and get KeyManagers
+ KeyStore keyStore = null;
+ KeyManager[] keyManagers = null;
+ if (keyStorePath.isPresent()) {
+ char[] keyPassword = keyStorePassword.map(String::toCharArray).orElse(null);
+
+ keyStore = KeyStore.getInstance(KeyStore.getDefaultType());
+ try (InputStream in = new FileInputStream(keyStorePath.get())) {
+ keyStore.load(in, keyPassword);
+ }
+
+ KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+ keyManagerFactory.init(keyStore, keyPassword);
+ keyManagers = keyManagerFactory.getKeyManagers();
+ }
+
+ // load TrustStore if configured, otherwise use KeyStore
+ KeyStore trustStore = keyStore;
+ if (trustStorePath.isPresent()) {
+ trustStore = KeyStore.getInstance(KeyStore.getDefaultType());
+ try (InputStream in = new FileInputStream(trustStorePath.get())) {
+ trustStore.load(in, trustStorePassword.map(String::toCharArray).orElse(null));
+ }
+ }
+
+ // create TrustManagerFactory
+ TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
+ trustManagerFactory.init(trustStore);
+
+ // get X509TrustManager
+ TrustManager[] trustManagers = trustManagerFactory.getTrustManagers();
+ if ((trustManagers.length != 1) || !(trustManagers[0] instanceof X509TrustManager)) {
+ throw new RuntimeException("Unexpected default trust managers:" + Arrays.toString(trustManagers));
+ }
+ X509TrustManager trustManager = (X509TrustManager) trustManagers[0];
+
+ // create SSLContext
+ SSLContext sslContext = SSLContext.getInstance("TLS");
+ sslContext.init(keyManagers, new TrustManager[] {trustManager}, null);
+
+ clientBuilder.sslSocketFactory(sslContext.getSocketFactory(), trustManager);
+ }
+ catch (GeneralSecurityException | IOException e) {
+ throw new ClientException("Error setting up SSL: " + e.getMessage(), e);
+ }
+ }
+
+ public static void setupKerberos(
+ OkHttpClient.Builder clientBuilder,
+ String remoteServiceName,
+ boolean useCanonicalHostname,
+ Optional principal,
+ Optional kerberosConfig,
+ Optional keytab,
+ Optional credentialCache)
+ {
+ SpnegoHandler handler = new SpnegoHandler(
+ remoteServiceName,
+ useCanonicalHostname,
+ principal,
+ kerberosConfig,
+ keytab,
+ credentialCache);
+ clientBuilder.addInterceptor(handler);
+ clientBuilder.authenticator(handler);
+ }
+}
diff --git a/presto-client/src/main/java/com/facebook/presto/client/ServerInfo.java b/presto-client/src/main/java/com/facebook/presto/client/ServerInfo.java
index 01207a920709d..30b039215e3fd 100644
--- a/presto-client/src/main/java/com/facebook/presto/client/ServerInfo.java
+++ b/presto-client/src/main/java/com/facebook/presto/client/ServerInfo.java
@@ -15,10 +15,12 @@
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
+import io.airlift.units.Duration;
import javax.annotation.concurrent.Immutable;
import java.util.Objects;
+import java.util.Optional;
import static com.google.common.base.MoreObjects.toStringHelper;
import static java.util.Objects.requireNonNull;
@@ -30,15 +32,20 @@ public class ServerInfo
private final String environment;
private final boolean coordinator;
+ // optional to maintain compatibility with older servers
+ private final Optional uptime;
+
@JsonCreator
public ServerInfo(
@JsonProperty("nodeVersion") NodeVersion nodeVersion,
@JsonProperty("environment") String environment,
- @JsonProperty("coordinator") boolean coordinator)
+ @JsonProperty("coordinator") boolean coordinator,
+ @JsonProperty("uptime") Optional uptime)
{
this.nodeVersion = requireNonNull(nodeVersion, "nodeVersion is null");
this.environment = requireNonNull(environment, "environment is null");
this.coordinator = requireNonNull(coordinator, "coordinator is null");
+ this.uptime = requireNonNull(uptime, "uptime is null");
}
@JsonProperty
@@ -59,6 +66,12 @@ public boolean isCoordinator()
return coordinator;
}
+ @JsonProperty
+ public Optional getUptime()
+ {
+ return uptime;
+ }
+
@Override
public boolean equals(Object o)
{
@@ -87,6 +100,8 @@ public String toString()
.add("nodeVersion", nodeVersion)
.add("environment", environment)
.add("coordinator", coordinator)
+ .add("uptime", uptime.orElse(null))
+ .omitNullValues()
.toString();
}
}
diff --git a/presto-client/src/main/java/com/facebook/presto/client/SpnegoHandler.java b/presto-client/src/main/java/com/facebook/presto/client/SpnegoHandler.java
new file mode 100644
index 0000000000000..cb08cb598392c
--- /dev/null
+++ b/presto-client/src/main/java/com/facebook/presto/client/SpnegoHandler.java
@@ -0,0 +1,332 @@
+/*
+ * 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.client;
+
+import com.google.common.base.Splitter;
+import com.google.common.collect.ImmutableMap;
+import com.sun.security.auth.module.Krb5LoginModule;
+import io.airlift.units.Duration;
+import okhttp3.Authenticator;
+import okhttp3.Interceptor;
+import okhttp3.Request;
+import okhttp3.Response;
+import okhttp3.Route;
+import org.ietf.jgss.GSSContext;
+import org.ietf.jgss.GSSCredential;
+import org.ietf.jgss.GSSException;
+import org.ietf.jgss.GSSManager;
+import org.ietf.jgss.Oid;
+
+import javax.annotation.concurrent.GuardedBy;
+import javax.security.auth.Subject;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.Configuration;
+import javax.security.auth.login.LoginContext;
+import javax.security.auth.login.LoginException;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.security.Principal;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import java.util.Base64;
+import java.util.Locale;
+import java.util.Optional;
+
+import static com.google.common.base.CharMatcher.whitespace;
+import static com.google.common.base.Throwables.throwIfInstanceOf;
+import static com.google.common.base.Throwables.throwIfUnchecked;
+import static com.google.common.net.HttpHeaders.AUTHORIZATION;
+import static com.google.common.net.HttpHeaders.WWW_AUTHENTICATE;
+import static java.lang.Boolean.getBoolean;
+import static java.lang.String.format;
+import static java.util.Objects.requireNonNull;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag.REQUIRED;
+import static org.ietf.jgss.GSSContext.INDEFINITE_LIFETIME;
+import static org.ietf.jgss.GSSCredential.DEFAULT_LIFETIME;
+import static org.ietf.jgss.GSSCredential.INITIATE_ONLY;
+import static org.ietf.jgss.GSSName.NT_HOSTBASED_SERVICE;
+import static org.ietf.jgss.GSSName.NT_USER_NAME;
+
+// TODO: This class is similar to SpnegoAuthentication in Airlift. Consider extracting a library.
+public class SpnegoHandler
+ implements Interceptor, Authenticator
+{
+ private static final String NEGOTIATE = "Negotiate";
+ private static final Duration MIN_CREDENTIAL_LIFETIME = new Duration(60, SECONDS);
+
+ private static final GSSManager GSS_MANAGER = GSSManager.getInstance();
+
+ private static final Oid SPNEGO_OID = createOid("1.3.6.1.5.5.2");
+ private static final Oid KERBEROS_OID = createOid("1.2.840.113554.1.2.2");
+
+ private final String remoteServiceName;
+ private final boolean useCanonicalHostname;
+ private final Optional principal;
+ private final Optional keytab;
+ private final Optional credentialCache;
+
+ @GuardedBy("this")
+ private Session clientSession;
+
+ public SpnegoHandler(
+ String remoteServiceName,
+ boolean useCanonicalHostname,
+ Optional principal,
+ Optional kerberosConfig,
+ Optional keytab,
+ Optional credentialCache)
+ {
+ this.remoteServiceName = requireNonNull(remoteServiceName, "remoteServiceName is null");
+ this.useCanonicalHostname = useCanonicalHostname;
+ this.principal = requireNonNull(principal, "principal is null");
+ this.keytab = requireNonNull(keytab, "keytab is null");
+ this.credentialCache = requireNonNull(credentialCache, "credentialCache is null");
+
+ kerberosConfig.ifPresent(file -> System.setProperty("java.security.krb5.conf", file.getAbsolutePath()));
+ }
+
+ @Override
+ public Response intercept(Chain chain)
+ throws IOException
+ {
+ // eagerly send authentication if possible
+ try {
+ return chain.proceed(authenticate(chain.request()));
+ }
+ catch (ClientException ignored) {
+ return chain.proceed(chain.request());
+ }
+ }
+
+ @Override
+ public Request authenticate(Route route, Response response)
+ throws IOException
+ {
+ // skip if we already tried or were not asked for Kerberos
+ if (response.request().headers(AUTHORIZATION).stream().anyMatch(SpnegoHandler::isNegotiate) ||
+ response.headers(WWW_AUTHENTICATE).stream().noneMatch(SpnegoHandler::isNegotiate)) {
+ return null;
+ }
+
+ return authenticate(response.request());
+ }
+
+ private static boolean isNegotiate(String value)
+ {
+ return Splitter.on(whitespace()).split(value).iterator().next().equalsIgnoreCase(NEGOTIATE);
+ }
+
+ private Request authenticate(Request request)
+ {
+ String hostName = request.url().host();
+ String principal = makeServicePrincipal(remoteServiceName, hostName, useCanonicalHostname);
+ byte[] token = generateToken(principal);
+
+ String credential = format("%s %s", NEGOTIATE, Base64.getEncoder().encodeToString(token));
+ return request.newBuilder()
+ .header(AUTHORIZATION, credential)
+ .build();
+ }
+
+ private byte[] generateToken(String servicePrincipal)
+ {
+ GSSContext context = null;
+ try {
+ Session session = getSession();
+ context = doAs(session.getLoginContext().getSubject(), () -> {
+ GSSContext result = GSS_MANAGER.createContext(
+ GSS_MANAGER.createName(servicePrincipal, NT_HOSTBASED_SERVICE),
+ SPNEGO_OID,
+ session.getClientCredential(),
+ INDEFINITE_LIFETIME);
+
+ result.requestMutualAuth(true);
+ result.requestConf(true);
+ result.requestInteg(true);
+ result.requestCredDeleg(false);
+ return result;
+ });
+
+ byte[] token = context.initSecContext(new byte[0], 0, 0);
+ if (token == null) {
+ throw new LoginException("No token generated from GSS context");
+ }
+ return token;
+ }
+ catch (GSSException | LoginException e) {
+ throw new ClientException(format("Kerberos error for [%s]: %s", servicePrincipal, e.getMessage()), e);
+ }
+ finally {
+ try {
+ if (context != null) {
+ context.dispose();
+ }
+ }
+ catch (GSSException ignored) {
+ }
+ }
+ }
+
+ private synchronized Session getSession()
+ throws LoginException, GSSException
+ {
+ if ((clientSession == null) || clientSession.needsRefresh()) {
+ clientSession = createSession();
+ }
+ return clientSession;
+ }
+
+ private Session createSession()
+ throws LoginException, GSSException
+ {
+ // TODO: do we need to call logout() on the LoginContext?
+
+ LoginContext loginContext = new LoginContext("", null, null, new Configuration()
+ {
+ @Override
+ public AppConfigurationEntry[] getAppConfigurationEntry(String name)
+ {
+ ImmutableMap.Builder options = ImmutableMap.builder();
+ options.put("refreshKrb5Config", "true");
+ options.put("doNotPrompt", "true");
+ options.put("useKeyTab", "true");
+
+ if (getBoolean("presto.client.debugKerberos")) {
+ options.put("debug", "true");
+ }
+
+ keytab.ifPresent(file -> options.put("keyTab", file.getAbsolutePath()));
+
+ credentialCache.ifPresent(file -> {
+ options.put("ticketCache", file.getAbsolutePath());
+ options.put("useTicketCache", "true");
+ options.put("renewTGT", "true");
+ });
+
+ principal.ifPresent(value -> options.put("principal", value));
+
+ return new AppConfigurationEntry[] {
+ new AppConfigurationEntry(Krb5LoginModule.class.getName(), REQUIRED, options.build())
+ };
+ }
+ });
+
+ loginContext.login();
+ Subject subject = loginContext.getSubject();
+ Principal clientPrincipal = subject.getPrincipals().iterator().next();
+ GSSCredential clientCredential = doAs(subject, () -> GSS_MANAGER.createCredential(
+ GSS_MANAGER.createName(clientPrincipal.getName(), NT_USER_NAME),
+ DEFAULT_LIFETIME,
+ KERBEROS_OID,
+ INITIATE_ONLY));
+
+ return new Session(loginContext, clientCredential);
+ }
+
+ private static String makeServicePrincipal(String serviceName, String hostName, boolean useCanonicalHostname)
+ {
+ String serviceHostName = hostName;
+ if (useCanonicalHostname) {
+ serviceHostName = canonicalizeServiceHostName(hostName);
+ }
+ return format("%s@%s", serviceName, serviceHostName.toLowerCase(Locale.US));
+ }
+
+ private static String canonicalizeServiceHostName(String hostName)
+ {
+ try {
+ InetAddress address = InetAddress.getByName(hostName);
+ String fullHostName;
+ if ("localhost".equalsIgnoreCase(address.getHostName())) {
+ fullHostName = InetAddress.getLocalHost().getCanonicalHostName();
+ }
+ else {
+ fullHostName = address.getCanonicalHostName();
+ }
+ if (fullHostName.equalsIgnoreCase("localhost")) {
+ throw new ClientException("Fully qualified name of localhost should not resolve to 'localhost'. System configuration error?");
+ }
+ return fullHostName;
+ }
+ catch (UnknownHostException e) {
+ throw new ClientException("Failed to resolve host: " + hostName, e);
+ }
+ }
+
+ private interface GssSupplier
+ {
+ T get()
+ throws GSSException;
+ }
+
+ private static T doAs(Subject subject, GssSupplier action)
+ throws GSSException
+ {
+ try {
+ return Subject.doAs(subject, (PrivilegedExceptionAction) action::get);
+ }
+ catch (PrivilegedActionException e) {
+ Throwable t = e.getCause();
+ throwIfInstanceOf(t, GSSException.class);
+ throwIfUnchecked(t);
+ throw new RuntimeException(t);
+ }
+ }
+
+ private static Oid createOid(String value)
+ {
+ try {
+ return new Oid(value);
+ }
+ catch (GSSException e) {
+ throw new AssertionError(e);
+ }
+ }
+
+ private static class Session
+ {
+ private final LoginContext loginContext;
+ private final GSSCredential clientCredential;
+
+ public Session(LoginContext loginContext, GSSCredential clientCredential)
+ throws LoginException
+ {
+ requireNonNull(loginContext, "loginContext is null");
+ requireNonNull(clientCredential, "gssCredential is null");
+
+ this.loginContext = loginContext;
+ this.clientCredential = clientCredential;
+ }
+
+ public LoginContext getLoginContext()
+ {
+ return loginContext;
+ }
+
+ public GSSCredential getClientCredential()
+ {
+ return clientCredential;
+ }
+
+ public boolean needsRefresh()
+ throws GSSException
+ {
+ return clientCredential.getRemainingLifetime() < MIN_CREDENTIAL_LIFETIME.getValue(SECONDS);
+ }
+ }
+}
diff --git a/presto-client/src/main/java/com/facebook/presto/client/StatementClient.java b/presto-client/src/main/java/com/facebook/presto/client/StatementClient.java
index 7373f17338af9..785a73b9ae38c 100644
--- a/presto-client/src/main/java/com/facebook/presto/client/StatementClient.java
+++ b/presto-client/src/main/java/com/facebook/presto/client/StatementClient.java
@@ -13,19 +13,19 @@
*/
package com.facebook.presto.client;
+import com.facebook.presto.client.OkHttpUtil.NullCallback;
import com.facebook.presto.spi.type.TimeZoneKey;
import com.google.common.base.Splitter;
-import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Sets;
-import io.airlift.http.client.FullJsonResponseHandler;
-import io.airlift.http.client.HttpClient;
-import io.airlift.http.client.HttpClient.HttpResponseFuture;
-import io.airlift.http.client.HttpStatus;
-import io.airlift.http.client.Request;
import io.airlift.json.JsonCodec;
-import io.airlift.units.Duration;
+import okhttp3.Headers;
+import okhttp3.HttpUrl;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
import javax.annotation.concurrent.ThreadSafe;
@@ -37,35 +37,36 @@
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
+import java.util.Optional;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import static com.facebook.presto.client.PrestoHeaders.PRESTO_ADDED_PREPARE;
+import static com.facebook.presto.client.PrestoHeaders.PRESTO_CATALOG;
import static com.facebook.presto.client.PrestoHeaders.PRESTO_CLEAR_SESSION;
import static com.facebook.presto.client.PrestoHeaders.PRESTO_CLEAR_TRANSACTION_ID;
+import static com.facebook.presto.client.PrestoHeaders.PRESTO_CLIENT_INFO;
import static com.facebook.presto.client.PrestoHeaders.PRESTO_DEALLOCATED_PREPARE;
+import static com.facebook.presto.client.PrestoHeaders.PRESTO_LANGUAGE;
+import static com.facebook.presto.client.PrestoHeaders.PRESTO_PREPARED_STATEMENT;
+import static com.facebook.presto.client.PrestoHeaders.PRESTO_SCHEMA;
+import static com.facebook.presto.client.PrestoHeaders.PRESTO_SESSION;
import static com.facebook.presto.client.PrestoHeaders.PRESTO_SET_SESSION;
+import static com.facebook.presto.client.PrestoHeaders.PRESTO_SOURCE;
import static com.facebook.presto.client.PrestoHeaders.PRESTO_STARTED_TRANSACTION_ID;
+import static com.facebook.presto.client.PrestoHeaders.PRESTO_TIME_ZONE;
+import static com.facebook.presto.client.PrestoHeaders.PRESTO_TRANSACTION_ID;
+import static com.facebook.presto.client.PrestoHeaders.PRESTO_USER;
import static com.google.common.base.MoreObjects.firstNonNull;
import static com.google.common.base.Preconditions.checkState;
import static com.google.common.net.HttpHeaders.USER_AGENT;
-import static io.airlift.http.client.FullJsonResponseHandler.JsonResponse;
-import static io.airlift.http.client.FullJsonResponseHandler.createFullJsonResponseHandler;
-import static io.airlift.http.client.HttpStatus.Family;
-import static io.airlift.http.client.HttpStatus.familyForStatusCode;
-import static io.airlift.http.client.HttpUriBuilder.uriBuilderFrom;
-import static io.airlift.http.client.Request.Builder.prepareDelete;
-import static io.airlift.http.client.Request.Builder.prepareGet;
-import static io.airlift.http.client.Request.Builder.preparePost;
-import static io.airlift.http.client.StaticBodyGenerator.createStaticBodyGenerator;
-import static io.airlift.http.client.StatusResponseHandler.StatusResponse;
-import static io.airlift.http.client.StatusResponseHandler.createStatusResponseHandler;
+import static io.airlift.json.JsonCodec.jsonCodec;
import static java.lang.String.format;
-import static java.nio.charset.StandardCharsets.UTF_8;
+import static java.net.HttpURLConnection.HTTP_OK;
+import static java.net.HttpURLConnection.HTTP_UNAUTHORIZED;
+import static java.net.HttpURLConnection.HTTP_UNAVAILABLE;
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.NANOSECONDS;
@@ -74,13 +75,15 @@
public class StatementClient
implements Closeable
{
+ private static final MediaType MEDIA_TYPE_JSON = MediaType.parse("application/json; charset=utf-8");
+ private static final JsonCodec QUERY_RESULTS_CODEC = jsonCodec(QueryResults.class);
+
private static final Splitter SESSION_HEADER_SPLITTER = Splitter.on('=').limit(2).trimResults();
private static final String USER_AGENT_VALUE = StatementClient.class.getSimpleName() +
"/" +
firstNonNull(StatementClient.class.getPackage().getImplementationVersion(), "unknown");
- private final HttpClient httpClient;
- private final FullJsonResponseHandler responseHandler;
+ private final OkHttpClient httpClient;
private final boolean debug;
private final String query;
private final AtomicReference currentResults = new AtomicReference<>();
@@ -97,15 +100,13 @@ public class StatementClient
private final long requestTimeoutNanos;
private final String user;
- public StatementClient(HttpClient httpClient, JsonCodec queryResultsCodec, ClientSession session, String query)
+ public StatementClient(OkHttpClient httpClient, ClientSession session, String query)
{
requireNonNull(httpClient, "httpClient is null");
- requireNonNull(queryResultsCodec, "queryResultsCodec is null");
requireNonNull(session, "session is null");
requireNonNull(query, "query is null");
this.httpClient = httpClient;
- this.responseHandler = createFullJsonResponseHandler(queryResultsCodec);
this.debug = session.isDebug();
this.timeZone = session.getTimeZone();
this.query = query;
@@ -113,48 +114,54 @@ public StatementClient(HttpClient httpClient, JsonCodec queryResul
this.user = session.getUser();
Request request = buildQueryRequest(session, query);
- JsonResponse response = httpClient.execute(request, responseHandler);
- if (response.getStatusCode() != HttpStatus.OK.code() || !response.hasValue()) {
+ JsonResponse response = JsonResponse.execute(QUERY_RESULTS_CODEC, httpClient, request);
+ if ((response.getStatusCode() != HTTP_OK) || !response.hasValue()) {
throw requestFailedException("starting query", request, response);
}
- processResponse(response);
+ processResponse(response.getHeaders(), response.getValue());
}
private Request buildQueryRequest(ClientSession session, String query)
{
- Request.Builder builder = prepareRequest(preparePost(), uriBuilderFrom(session.getServer()).replacePath("/v1/statement").build())
- .setBodyGenerator(createStaticBodyGenerator(query, UTF_8));
+ HttpUrl url = HttpUrl.get(session.getServer());
+ if (url == null) {
+ throw new ClientException("Invalid server URL: " + session.getServer());
+ }
+ url = url.newBuilder().encodedPath("/v1/statement").build();
+
+ Request.Builder builder = prepareRequest(url)
+ .post(RequestBody.create(MEDIA_TYPE_JSON, query));
if (session.getSource() != null) {
- builder.setHeader(PrestoHeaders.PRESTO_SOURCE, session.getSource());
+ builder.addHeader(PRESTO_SOURCE, session.getSource());
}
if (session.getClientInfo() != null) {
- builder.setHeader(PrestoHeaders.PRESTO_CLIENT_INFO, session.getClientInfo());
+ builder.addHeader(PRESTO_CLIENT_INFO, session.getClientInfo());
}
if (session.getCatalog() != null) {
- builder.setHeader(PrestoHeaders.PRESTO_CATALOG, session.getCatalog());
+ builder.addHeader(PRESTO_CATALOG, session.getCatalog());
}
if (session.getSchema() != null) {
- builder.setHeader(PrestoHeaders.PRESTO_SCHEMA, session.getSchema());
+ builder.addHeader(PRESTO_SCHEMA, session.getSchema());
}
- builder.setHeader(PrestoHeaders.PRESTO_TIME_ZONE, session.getTimeZone().getId());
+ builder.addHeader(PRESTO_TIME_ZONE, session.getTimeZone().getId());
if (session.getLocale() != null) {
- builder.setHeader(PrestoHeaders.PRESTO_LANGUAGE, session.getLocale().toLanguageTag());
+ builder.addHeader(PRESTO_LANGUAGE, session.getLocale().toLanguageTag());
}
Map property = session.getProperties();
for (Entry entry : property.entrySet()) {
- builder.addHeader(PrestoHeaders.PRESTO_SESSION, entry.getKey() + "=" + entry.getValue());
+ builder.addHeader(PRESTO_SESSION, entry.getKey() + "=" + entry.getValue());
}
Map statements = session.getPreparedStatements();
for (Entry entry : statements.entrySet()) {
- builder.addHeader(PrestoHeaders.PRESTO_PREPARED_STATEMENT, urlEncode(entry.getKey()) + "=" + urlEncode(entry.getValue()));
+ builder.addHeader(PRESTO_PREPARED_STATEMENT, urlEncode(entry.getKey()) + "=" + urlEncode(entry.getValue()));
}
- builder.setHeader(PrestoHeaders.PRESTO_TRANSACTION_ID, session.getTransactionId() == null ? "NONE" : session.getTransactionId());
+ builder.addHeader(PRESTO_TRANSACTION_ID, session.getTransactionId() == null ? "NONE" : session.getTransactionId());
return builder.build();
}
@@ -241,13 +248,12 @@ public boolean isValid()
return valid.get() && (!isGone()) && (!isClosed());
}
- private Request.Builder prepareRequest(Request.Builder builder, URI nextUri)
+ private Request.Builder prepareRequest(HttpUrl url)
{
- builder.setHeader(PrestoHeaders.PRESTO_USER, user);
- builder.setHeader(USER_AGENT, USER_AGENT_VALUE)
- .setUri(nextUri);
-
- return builder;
+ return new Request.Builder()
+ .addHeader(PRESTO_USER, user)
+ .addHeader(USER_AGENT, USER_AGENT_VALUE)
+ .url(url);
}
public boolean advance()
@@ -258,7 +264,7 @@ public boolean advance()
return false;
}
- Request request = prepareRequest(prepareGet(), nextUri).build();
+ Request request = prepareRequest(HttpUrl.get(nextUri)).build();
Exception cause = null;
long start = System.nanoTime();
@@ -284,19 +290,19 @@ public boolean advance()
JsonResponse response;
try {
- response = httpClient.execute(request, responseHandler);
+ response = JsonResponse.execute(QUERY_RESULTS_CODEC, httpClient, request);
}
catch (RuntimeException e) {
cause = e;
continue;
}
- if (response.getStatusCode() == HttpStatus.OK.code() && response.hasValue()) {
- processResponse(response);
+ if ((response.getStatusCode() == HTTP_OK) && response.hasValue()) {
+ processResponse(response.getHeaders(), response.getValue());
return true;
}
- if (response.getStatusCode() != HttpStatus.SERVICE_UNAVAILABLE.code()) {
+ if (response.getStatusCode() != HTTP_UNAVAILABLE) {
throw requestFailedException("fetching next", request, response);
}
}
@@ -306,77 +312,65 @@ public boolean advance()
throw new RuntimeException("Error fetching next", cause);
}
- private void processResponse(JsonResponse response)
+ private void processResponse(Headers headers, QueryResults results)
{
- for (String setSession : response.getHeaders(PRESTO_SET_SESSION)) {
+ for (String setSession : headers.values(PRESTO_SET_SESSION)) {
List keyValue = SESSION_HEADER_SPLITTER.splitToList(setSession);
if (keyValue.size() != 2) {
continue;
}
setSessionProperties.put(keyValue.get(0), keyValue.size() > 1 ? keyValue.get(1) : "");
}
- for (String clearSession : response.getHeaders(PRESTO_CLEAR_SESSION)) {
+ for (String clearSession : headers.values(PRESTO_CLEAR_SESSION)) {
resetSessionProperties.add(clearSession);
}
- for (String entry : response.getHeaders(PRESTO_ADDED_PREPARE)) {
+ for (String entry : headers.values(PRESTO_ADDED_PREPARE)) {
List keyValue = SESSION_HEADER_SPLITTER.splitToList(entry);
if (keyValue.size() != 2) {
continue;
}
addedPreparedStatements.put(urlDecode(keyValue.get(0)), urlDecode(keyValue.get(1)));
}
- for (String entry : response.getHeaders(PRESTO_DEALLOCATED_PREPARE)) {
+ for (String entry : headers.values(PRESTO_DEALLOCATED_PREPARE)) {
deallocatedPreparedStatements.add(urlDecode(entry));
}
- String startedTransactionId = response.getHeader(PRESTO_STARTED_TRANSACTION_ID);
+ String startedTransactionId = headers.get(PRESTO_STARTED_TRANSACTION_ID);
if (startedTransactionId != null) {
this.startedtransactionId.set(startedTransactionId);
}
- if (response.getHeader(PRESTO_CLEAR_TRANSACTION_ID) != null) {
+ if (headers.values(PRESTO_CLEAR_TRANSACTION_ID) != null) {
clearTransactionId.set(true);
}
- currentResults.set(response.getValue());
+ currentResults.set(results);
}
private RuntimeException requestFailedException(String task, Request request, JsonResponse response)
{
gone.set(true);
if (!response.hasValue()) {
+ if (response.getStatusCode() == HTTP_UNAUTHORIZED) {
+ return new ClientException("Authentication failed" +
+ Optional.ofNullable(response.getStatusMessage())
+ .map(message -> ": " + message)
+ .orElse(""));
+ }
return new RuntimeException(
- format("Error %s at %s returned an invalid response: %s [Error: %s]", task, request.getUri(), response, response.getResponseBody()),
+ format("Error %s at %s returned an invalid response: %s [Error: %s]", task, request.url(), response, response.getResponseBody()),
response.getException());
}
- return new RuntimeException(format("Error %s at %s returned %s: %s", task, request.getUri(), response.getStatusCode(), response.getStatusMessage()));
+ return new RuntimeException(format("Error %s at %s returned HTTP %s", task, request.url(), response.getStatusCode()));
}
- public boolean cancelLeafStage(Duration timeout)
+ public void cancelLeafStage()
{
checkState(!isClosed(), "client is closed");
URI uri = current().getPartialCancelUri();
- if (uri == null) {
- return false;
- }
-
- Request request = prepareRequest(prepareDelete(), uri).build();
-
- HttpResponseFuture response = httpClient.executeAsync(request, createStatusResponseHandler());
- try {
- StatusResponse status = response.get(timeout.toMillis(), MILLISECONDS);
- return familyForStatusCode(status.getStatusCode()) == Family.SUCCESSFUL;
- }
- catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- throw Throwables.propagate(e);
- }
- catch (ExecutionException e) {
- throw Throwables.propagate(e.getCause());
- }
- catch (TimeoutException e) {
- return false;
+ if (uri != null) {
+ httpDelete(uri);
}
}
@@ -386,12 +380,19 @@ public void close()
if (!closed.getAndSet(true)) {
URI uri = currentResults.get().getNextUri();
if (uri != null) {
- Request request = prepareRequest(prepareDelete(), uri).build();
- httpClient.executeAsync(request, createStatusResponseHandler());
+ httpDelete(uri);
}
}
}
+ private void httpDelete(URI uri)
+ {
+ Request request = prepareRequest(HttpUrl.get(uri))
+ .delete()
+ .build();
+ httpClient.newCall(request).enqueue(new NullCallback());
+ }
+
private static String urlEncode(String value)
{
try {
diff --git a/presto-client/src/test/java/com/facebook/presto/client/TestServerInfo.java b/presto-client/src/test/java/com/facebook/presto/client/TestServerInfo.java
new file mode 100644
index 0000000000000..036851d4252a8
--- /dev/null
+++ b/presto-client/src/test/java/com/facebook/presto/client/TestServerInfo.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.facebook.presto.client;
+
+import io.airlift.json.JsonCodec;
+import io.airlift.units.Duration;
+import org.testng.annotations.Test;
+
+import java.util.Optional;
+
+import static com.facebook.presto.client.NodeVersion.UNKNOWN;
+import static io.airlift.json.JsonCodec.jsonCodec;
+import static org.testng.Assert.assertEquals;
+
+public class TestServerInfo
+{
+ private static final JsonCodec SERVER_INFO_CODEC = jsonCodec(ServerInfo.class);
+
+ @Test
+ public void testJsonRoundTrip()
+ {
+ assertJsonRoundTrip(new ServerInfo(UNKNOWN, "test", true, Optional.of(Duration.valueOf("2m"))));
+ assertJsonRoundTrip(new ServerInfo(UNKNOWN, "test", true, Optional.empty()));
+ }
+
+ @Test
+ public void testBackwardsCompatible()
+ {
+ ServerInfo newServerInfo = new ServerInfo(UNKNOWN, "test", true, Optional.empty());
+ ServerInfo legacyServerInfo = SERVER_INFO_CODEC.fromJson("{\"nodeVersion\":{\"version\":\"\"},\"environment\":\"test\",\"coordinator\":true}");
+ assertEquals(newServerInfo, legacyServerInfo);
+ }
+
+ private static void assertJsonRoundTrip(ServerInfo serverInfo)
+ {
+ String json = SERVER_INFO_CODEC.toJson(serverInfo);
+ ServerInfo copy = SERVER_INFO_CODEC.fromJson(json);
+ assertEquals(copy, serverInfo);
+ }
+}
diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml
index 36d725071547c..ae09ccdb7e089 100644
--- a/presto-docs/pom.xml
+++ b/presto-docs/pom.xml
@@ -5,7 +5,7 @@
com.facebook.presto
presto-root
- 0.175-SNAPSHOT
+ 0.180-SNAPSHOT
presto-docs
@@ -44,6 +44,38 @@
+
+ org.codehaus.mojo
+ exec-maven-plugin
+
+
+ validate-reserved
+ validate
+
+ java
+
+
+ com.facebook.presto.sql.ReservedIdentifiers
+
+ validateDocs
+ ${project.basedir}/src/main/sphinx/language/reserved.rst
+
+
+
+
+
+ false
+ true
+
+
+
+ com.facebook.presto
+ presto-parser
+ ${project.version}
+
+
+
+
io.airlift.maven.plugins
sphinx-maven-plugin
diff --git a/presto-docs/src/main/sphinx/admin.rst b/presto-docs/src/main/sphinx/admin.rst
index eab299a0a3925..0af2c6e6a3c0b 100644
--- a/presto-docs/src/main/sphinx/admin.rst
+++ b/presto-docs/src/main/sphinx/admin.rst
@@ -7,5 +7,6 @@ Administration
admin/web-interface
admin/tuning
+ admin/properties
admin/queue
admin/resource-groups
diff --git a/presto-docs/src/main/sphinx/admin/properties.rst b/presto-docs/src/main/sphinx/admin/properties.rst
new file mode 100644
index 0000000000000..6284c52f07bf1
--- /dev/null
+++ b/presto-docs/src/main/sphinx/admin/properties.rst
@@ -0,0 +1,416 @@
+====================
+Properties Reference
+====================
+
+This section describes the most important config properties that
+may be used to tune Presto or alter its behavior when required.
+
+.. contents::
+ :local:
+ :backlinks: none
+ :depth: 1
+
+General Properties
+------------------
+
+``distributed-joins-enabled``
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+ * **Type:** ``boolean``
+ * **Default value:** ``true``
+
+ Use hash distributed joins instead of broadcast joins. Distributed joins
+ require redistributing both tables using a hash of the join key. This can
+ be slower (sometimes substantially) than broadcast joins, but allows much
+ larger joins. Broadcast joins require that the tables on the right side of
+ the join after filtering fit in memory on each node, whereas distributed joins
+ only need to fit in distributed memory across all nodes. This can also be
+ specified on a per-query basis using the ``distributed_join`` session property.
+
+``redistribute-writes``
+^^^^^^^^^^^^^^^^^^^^^^^
+
+ * **Type:** ``boolean``
+ * **Default value:** ``true``
+
+ This property enables redistribution of data before writing. This can
+ eliminate the performance impact of data skew when writing by hashing it
+ across nodes in the cluster. It can be disabled when it is known that the
+ output data set is not skewed in order to avoid the overhead of hashing and
+ redistributing all the data across the network. This can also be specified
+ on a per-query basis using the ``redistribute_writes`` session property.
+
+``resources.reserved-system-memory``
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+ * **Type:** ``data size``
+ * **Default value:** ``JVM max memory * 0.4``
+
+ The amount of JVM memory reserved, for accounting purposes, for things
+ that are not directly attributable to or controllable by a user query.
+ For example, output buffers, code caches, etc. This also accounts for
+ memory that is not tracked by the memory tracking system.
+
+ The purpose of this property is to prevent the JVM from running out of
+ memory (OOM). The default value is suitable for smaller JVM heap sizes or
+ clusters with many concurrent queries. If running fewer queries with a
+ large heap, a smaller value may work. Basically, set this value large
+ enough that the JVM does not fail with ``OutOfMemoryError``.
+
+
+Exchange Properties
+-------------------
+
+Exchanges transfer data between Presto nodes for different stages of
+a query. Adjusting these properties may help to resolve inter-node
+communication issues or improve network utilization.
+
+``exchange.client-threads``
+^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+ * **Type:** ``integer``
+ * **Minimum value:** ``1``
+ * **Default value:** ``25``
+
+ Number of threads used by exchange clients to fetch data from other Presto
+ nodes. A higher value can improve performance for large clusters or clusters
+ with very high concurrency, but excessively high values may cause a drop
+ in performance due to context switches and additional memory usage.
+
+``exchange.concurrent-request-multiplier``
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+ * **Type:** ``integer``
+ * **Minimum value:** ``1``
+ * **Default value:** ``3``
+
+ Multiplier determining the number of concurrent requests relative to
+ available buffer memory. The maximum number of requests is determined
+ using a heuristic of the number of clients that can fit into available
+ buffer space based on average buffer usage per request times this
+ multiplier. For example, with an ``exchange.max-buffer-size`` of ``32 MB``
+ and ``20 MB`` already used and average size per request being ``2MB``,
+ the maximum number of clients is
+ ``multiplier * ((32MB - 20MB) / 2MB) = multiplier * 6``. Tuning this
+ value adjusts the heuristic, which may increase concurrency and improve
+ network utilization.
+
+``exchange.max-buffer-size``
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+ * **Type:** ``data size``
+ * **Default value:** ``32MB``
+
+ Size of buffer in the exchange client that holds data fetched from other
+ nodes before it is processed. A larger buffer can increase network
+ throughput for larger clusters and thus decrease query processing time,
+ but will reduce the amount of memory available for other usages.
+
+``exchange.max-response-size``
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+ * **Type:** ``data size``
+ * **Minimum value:** ``1MB``
+ * **Default value:** ``16MB``
+
+ Maximum size of a response returned from an exchange request. The response
+ will be placed in the exchange client buffer which is shared across all
+ concurrent requests for the exchange.
+
+ Increasing the value may improve network throughput if there is high
+ latency. Decreasing the value may improve query performance for large
+ clusters as it reduces skew due to the exchange client buffer holding
+ responses for more tasks (rather than hold more data from fewer tasks).
+
+``sink.max-buffer-size``
+^^^^^^^^^^^^^^^^^^^^^^^^
+
+ * **Type:** ``data size``
+ * **Default value:** ``32MB``
+
+ Output buffer size for task data that is waiting to be pulled by upstream
+ tasks. If the task output is hash partitioned, then the buffer will be
+ shared across all of the partitioned consumers. Increasing this value may
+ improve network throughput for data transferred between stages if the
+ network has high latency or if there are many nodes in the cluster.
+
+
+Task Properties
+---------------
+
+``task.concurrency``
+^^^^^^^^^^^^^^^^^^^^
+
+ * **Type:** ``integer``
+ * **Restrictions:** must be a power of two
+ * **Default value:** ``16``
+
+ Default local concurrency for parallel operators such as joins and aggregations.
+ This value should be adjusted up or down based on the query concurrency and worker
+ resource utilization. Lower values are better for clusters that run many queries
+ concurrently because the cluster will already be utilized by all the running
+ queries, so adding more concurrency will result in slow downs due to context
+ switching and other overhead. Higher values are better for clusters that only run
+ one or a few queries at a time. This can also be specified on a per-query basis
+ using the ``task_concurrency`` session property.
+
+``task.http-response-threads``
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+ * **Type:** ``integer``
+ * **Minimum value:** ``1``
+ * **Default value:** ``100``
+
+ Maximum number of threads that may be created to handle HTTP responses. Threads are
+ created on demand and are cleaned up when idle, thus there is no overhead to a large
+ value if the number of requests to be handled is small. More threads may be helpful
+ on clusters with a high number of concurrent queries, or on clusters with hundreds
+ or thousands of workers.
+
+``task.http-timeout-threads``
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+ * **Type:** ``integer``
+ * **Minimum value:** ``1``
+ * **Default value:** ``3``
+
+ Number of threads used to handle timeouts when generating HTTP responses. This value
+ should be increased if all the threads are frequently in use. This can be monitored
+ via the ``com.facebook.presto.server:name=AsyncHttpExecutionMBean:TimeoutExecutor``
+ JMX object. If ``ActiveCount`` is always the same as ``PoolSize``, increase the
+ number of threads.
+
+``task.info-update-interval``
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+ * **Type:** ``duration``
+ * **Minimum value:** ``1ms``
+ * **Maximum value:** ``10s``
+ * **Default value:** ``3s``
+
+ Controls staleness of task information, which is used in scheduling. Larger values
+ can reduce coordinator CPU load, but may result in suboptimal split scheduling.
+
+``task.max-partial-aggregation-memory``
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+ * **Type:** ``data size``
+ * **Default value:** ``16MB``
+
+ Maximum size of partial aggregation results for distributed aggregations. Increasing this
+ value can result in less network transfer and lower CPU utilization by allowing more
+ groups to be kept locally before being flushed, at the cost of additional memory usage.
+
+``task.max-worker-threads``
+^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+ * **Type:** ``integer``
+ * **Default value:** ``Node CPUs * 2``
+
+ Sets the number of threads used by workers to process splits. Increasing this number
+ can improve throughput if worker CPU utilization is low and all the threads are in use,
+ but will cause increased heap space usage. Setting the value too high may cause a drop
+ in performance due to a context switching. The number of active threads is available
+ via the ``RunningSplits`` property of the
+ ``com.facebook.presto.execution.executor:name=TaskExecutor.RunningSplits`` JXM object.
+
+``task.min-drivers``
+^^^^^^^^^^^^^^^^^^^^
+
+ * **Type:** ``integer``
+ * **Default value:** ``task.max-worker-threads * 2``
+
+ The target number of running leaf splits on a worker. This is a minimum value because
+ each leaf task is guaranteed at least ``3`` running splits. Non-leaf tasks are also
+ guaranteed to run in order to prevent deadlocks. A lower value may improve responsiveness
+ for new tasks, but can result in underutilized resources. A higher value can increase
+ resource utilization, but uses additional memory.
+
+``task.writer-count``
+^^^^^^^^^^^^^^^^^^^^^
+
+ * **Type:** ``integer``
+ * **Restrictions:** must be a power of two
+ * **Default value:** ``1``
+
+ The number of concurrent writer threads per worker per query. Increasing this value may
+ increase write speed, especially when a query is not I/O bound and can take advantage
+ of additional CPU for parallel writes (some connectors can be bottlenecked on CPU when
+ writing due to compression or other factors). Setting this too high may cause the cluster
+ to become overloaded due to excessive resource utilization. This can also be specified on
+ a per-query basis using the ``task_writer_count`` session property.
+
+
+Node Scheduler Properties
+-------------------------
+
+``node-scheduler.max-splits-per-node``
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+ * **Type:** ``integer``
+ * **Default value:** ``100``
+
+ The target value for the total number of splits that can be running for
+ each worker node.
+
+ Using a higher value is recommended if queries are submitted in large batches
+ (e.g., running a large group of reports periodically) or for connectors that
+ produce many splits that complete quickly. Increasing this value may improve
+ query latency by ensuring that the workers have enough splits to keep them
+ fully utilized.
+
+ Setting this too high will waste memory and may result in lower performance
+ due to splits not being balanced across workers. Ideally, it should be set
+ such that there is always at least one split waiting to be processed, but
+ not higher.
+
+``node-scheduler.max-pending-splits-per-task``
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+ * **Type:** ``integer``
+ * **Default value:** ``10``
+
+ The number of outstanding splits that can be queued for each worker node
+ for a single stage of a query, even when the node is already at the limit for
+ total number of splits. Allowing a minimum number of splits per stage is
+ required to prevent starvation and deadlocks.
+
+ This value must be smaller than ``node-scheduler.max-splits-per-node``,
+ will usually be increased for the same reasons, and has similar drawbacks
+ if set too high.
+
+``node-scheduler.min-candidates``
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+ * **Type:** ``integer``
+ * **Minimum value:** ``1``
+ * **Default value:** ``10``
+
+ The minimum number of candidate nodes that will be evaluated by the
+ node scheduler when choosing the target node for a split. Setting
+ this value too low may prevent splits from being properly balanced
+ across all worker nodes. Setting it too high may increase query
+ latency and increase CPU usage on the coordinator.
+
+``node-scheduler.network-topology``
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+ * **Type:** ``string``
+ * **Allowed values:** ``legacy``, ``flat``
+ * **Default value:** ``legacy``
+
+
+Optimizer Properties
+--------------------
+
+``optimizer.dictionary-aggregation``
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+ * **Type:** ``boolean``
+ * **Default value:** ``false``
+
+ Enables optimization for aggregations on dictionaries. This can also be specified
+ on a per-query basis using the ``dictionary_aggregation`` session property.
+
+``optimizer.optimize-hash-generation``
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+ * **Type:** ``boolean``
+ * **Default value:** ``true``
+
+ Compute hash codes for distribution, joins, and aggregations early during execution,
+ allowing result to be shared between operations later in the query. This can reduce
+ CPU usage by avoiding computing the same hash multiple times, but at the cost of
+ additional network transfer for the hashes. In most cases it will decrease overall
+ query processing time. This can also be specified on a per-query basis using the
+ ``optimize_hash_generation`` session property.
+
+ It is often helpful to disable this property when using :doc:`/sql/explain` in order
+ to make the query plan easier to read.
+
+``optimizer.optimize-metadata-queries``
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+ * **Type:** ``boolean``
+ * **Default value:** ``false``
+
+ Enable optimization of some aggregations by using values that are stored as metadata.
+ This allows Presto to execute some simple queries in constant time. Currently, this
+ optimization applies to ``max``, ``min`` and ``approx_distinct`` of partition
+ keys and other aggregation insensitive to the cardinality of the input (including
+ ``DISTINCT`` aggregates). Using this may speed up some queries significantly.
+
+ The main drawback is that it can produce incorrect results if the connector returns
+ partition keys for partitions that have no rows. In particular, the Hive connector
+ can return empty partitions if they were created by other systems (Presto cannot
+ create them).
+
+``optimizer.optimize-single-distinct``
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+ * **Type:** ``boolean``
+ * **Default value:** ``true``
+
+ The single distinct optimization will try to replace multiple ``DISTINCT`` clauses
+ with a single ``GROUP BY`` clause, which can be substantially faster to execute.
+
+``optimizer.push-table-write-through-union``
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+ * **Type:** ``boolean``
+ * **Default value:** ``true``
+
+ Parallelize writes when using ``UNION ALL`` in queries that write data. This improves the
+ speed of writing output tables in ``UNION ALL`` queries because these writes do not require
+ additional synchronization when collecting results. Enabling this optimization can improve
+ ``UNION ALL`` speed when write speed is not yet saturated. However, it may slow down queries
+ in an already heavily loaded system. This can also be specified on a per-query basis
+ using the ``push_table_write_through_union`` session property.
+
+
+Regular Expression Function Properties
+--------------------------------------
+
+The following properties allow tuning the :doc:`/functions/regexp`.
+
+``regex-library``
+^^^^^^^^^^^^^^^^^
+
+ * **Type:** ``string``
+ * **Allowed values:** ``JONI``, ``RE2J``
+ * **Default value:** ``JONI``
+
+ Which library to use for regular expression functions.
+ ``JONI`` is generally faster for common usage, but can require exponential
+ time for certain expression patterns. ``RE2J`` uses a different algorithm
+ which guarantees linear time, but is often slower.
+
+``re2j.dfa-states-limit``
+^^^^^^^^^^^^^^^^^^^^^^^^^
+
+ * **Type:** ``integer``
+ * **Minimum value:** ``2``
+ * **Default value:** ``2147483647``
+
+ The maximum number of states to use when RE2J builds the fast
+ but potentially memory intensive deterministic finite automaton (DFA)
+ for regular expression matching. If the limit is reached, RE2J will fall
+ back to the algorithm that uses the slower, but less memory intensive
+ non-deterministic finite automaton (NFA). Decreasing this value decreases the
+ maximum memory footprint of a regular expression search at the cost of speed.
+
+``re2j.dfa-retries``
+^^^^^^^^^^^^^^^^^^^^
+
+ * **Type:** ``integer``
+ * **Minimum value:** ``0``
+ * **Default value:** ``5``
+
+ The number of times that RE2J will retry the DFA algorithm when
+ it reaches a states limit before using the slower, but less memory
+ intensive NFA algorithm for all future inputs for that search. If hitting the
+ limit for a given input row is likely to be an outlier, you want to be able
+ to process subsequent rows using the faster DFA algorithm. If you are likely
+ to hit the limit on matches for subsequent rows as well, you want to use the
+ correct algorithm from the beginning so as not to waste time and resources.
+ The more rows you are processing, the larger this value should be.
diff --git a/presto-docs/src/main/sphinx/admin/resource-groups.rst b/presto-docs/src/main/sphinx/admin/resource-groups.rst
index 2d61dff52d3ad..f17881b387cb0 100644
--- a/presto-docs/src/main/sphinx/admin/resource-groups.rst
+++ b/presto-docs/src/main/sphinx/admin/resource-groups.rst
@@ -96,7 +96,7 @@ There are three selectors that define which queries run in which resource group:
* The last selector is a catch all, which puts all queries into the user's adhoc group.
All together these selectors implement the policy that ``bob`` is an admin and
-all other users are subject to the follow limits:
+all other users are subject to the following limits:
* Users are allowed to have up to 2 adhoc queries running. Additionally, they may run one pipeline.
@@ -146,7 +146,7 @@ all other users are subject to the follow limits:
{
"name": "admin",
"softMemoryLimit": "100%",
- "maxRunning": 100,
+ "maxRunning": 200,
"maxQueued": 100,
"schedulingPolicy": "query_priority",
"jmxExport": true
diff --git a/presto-docs/src/main/sphinx/admin/tuning.rst b/presto-docs/src/main/sphinx/admin/tuning.rst
index 8d79b73ad1fdd..dabf0c25a7a5c 100644
--- a/presto-docs/src/main/sphinx/admin/tuning.rst
+++ b/presto-docs/src/main/sphinx/admin/tuning.rst
@@ -8,27 +8,7 @@ information may help you if your cluster is facing a specific performance proble
Config Properties
-----------------
-These configuration options may require tuning in specific situations:
-
-* ``task.max-worker-threads``:
- Sets the number of threads used by workers to process splits. Increasing this number
- can improve throughput if worker CPU utilization is low and all the threads are in use,
- but will cause increased heap space usage. The number of active threads is available via
- the ``com.facebook.presto.execution.executor.TaskExecutor.RunningSplits`` JMX stat.
-
-* ``distributed-joins-enabled``:
- Use hash distributed joins instead of broadcast joins. Distributed joins
- require redistributing both tables using a hash of the join key. This can
- be slower (sometimes substantially) than broadcast joins, but allows much
- larger joins. Broadcast joins require that the tables on the right side of
- the join fit in memory on each machine, whereas with distributed joins the
- tables on the right side have to fit in distributed memory. This can also be
- specified on a per-query basis using the ``distributed_join`` session property.
-
-* ``node-scheduler.network-topology``:
- Sets the network topology to use when scheduling splits. "legacy" will ignore
- the topology when scheduling splits. "flat" will try to schedule splits on the same
- host as the data is located by reserving 50% of the work queue for local splits.
+See :doc:`/admin/properties`.
JVM Settings
------------
diff --git a/presto-docs/src/main/sphinx/connector/accumulo.rst b/presto-docs/src/main/sphinx/connector/accumulo.rst
index 6c77836abf8c8..49af8f9eec17d 100644
--- a/presto-docs/src/main/sphinx/connector/accumulo.rst
+++ b/presto-docs/src/main/sphinx/connector/accumulo.rst
@@ -480,20 +480,22 @@ Note that session properties are prefixed with the catalog name::
SET SESSION accumulo.column_filter_optimizations_enabled = false;
-======================================== ============= =======================================================================================================
-Property Name Default Value Description
-======================================== ============= =======================================================================================================
-``optimize_locality_enabled`` ``true`` Set to true to enable data locality for non-indexed scans
-``optimize_split_ranges_enabled`` ``true`` Set to true to split non-indexed queries by tablet splits. Should generally be true.
-``optimize_index_enabled`` ``true`` Set to true to enable usage of the secondary index on query
-``index_rows_per_split`` ``10000`` The number of Accumulo row IDs that are packed into a single Presto split
-``index_threshold`` ``0.2`` The ratio between number of rows to be scanned based on the index over the total number of rows.
- If the ratio is below this threshold, the index will be used.
-``index_lowest_cardinality_threshold`` ``0.01`` The threshold where the column with the lowest cardinality will be used instead of computing an
- intersection of ranges in the index. Secondary index must be enabled.
-``index_metrics_enabled`` ``true`` Set to true to enable usage of the metrics table to optimize usage of the index
-``scan_username`` (config) User to impersonate when scanning the tables. This property trumps the ``scan_auths`` table property.
-======================================== ============= =======================================================================================================
+============================================= ============= =======================================================================================================
+Property Name Default Value Description
+============================================= ============= =======================================================================================================
+``optimize_locality_enabled`` ``true`` Set to true to enable data locality for non-indexed scans
+``optimize_split_ranges_enabled`` ``true`` Set to true to split non-indexed queries by tablet splits. Should generally be true.
+``optimize_index_enabled`` ``true`` Set to true to enable usage of the secondary index on query
+``index_rows_per_split`` ``10000`` The number of Accumulo row IDs that are packed into a single Presto split
+``index_threshold`` ``0.2`` The ratio between number of rows to be scanned based on the index over the total number of rows
+ If the ratio is below this threshold, the index will be used.
+``index_lowest_cardinality_threshold`` ``0.01`` The threshold where the column with the lowest cardinality will be used instead of computing an
+ intersection of ranges in the index. Secondary index must be enabled
+``index_metrics_enabled`` ``true`` Set to true to enable usage of the metrics table to optimize usage of the index
+``scan_username`` (config) User to impersonate when scanning the tables. This property trumps the ``scan_auths`` table property
+``index_short_circuit_cardinality_fetch`` ``true`` Short circuit the retrieval of index metrics once any column is less than the lowest cardinality threshold
+``index_cardinality_cache_polling_duration`` ``10ms`` Sets the cardinality cache polling duration for short circuit retrieval of index metrics
+============================================= ============= =======================================================================================================
Adding Columns
--------------
diff --git a/presto-docs/src/main/sphinx/connector/cassandra.rst b/presto-docs/src/main/sphinx/connector/cassandra.rst
index 13603af662028..9c93f5bbb4328 100644
--- a/presto-docs/src/main/sphinx/connector/cassandra.rst
+++ b/presto-docs/src/main/sphinx/connector/cassandra.rst
@@ -49,16 +49,6 @@ Property Name Description
``cassandra.native-protocol-port`` The Cassandra server port running the native client protocol
(defaults to ``9042``).
-``cassandra.max-schema-refresh-threads`` Maximum number of schema cache refresh threads. This property
- corresponds to the maximum number of parallel requests.
-
-``cassandra.schema-cache-ttl`` Maximum time that information about a schema will be cached
- (defaults to ``1h``).
-
-``cassandra.schema-refresh-interval`` The schema information cache will be refreshed in the background
- when accessed if the cached data is at least this old
- (defaults to ``2m``).
-
``cassandra.consistency-level`` Consistency levels in Cassandra refer to the level of consistency
to be used for both read and write operations. More information
about consistency levels can be found in the
@@ -184,3 +174,59 @@ This table can be described in Presto::
This table can then be queried in Presto::
SELECT * FROM cassandra.mykeyspace.users;
+
+Data types
+----------
+
+The data types mappings are as follows:
+
+================ ======
+Cassandra Presto
+================ ======
+ASCII VARCHAR
+BIGINT BIGINT
+BLOB VARBINARY
+BOOLEAN BOOLEAN
+DECIMAL DOUBLE
+DOUBLE DOUBLE
+FLOAT DOUBLE
+INET VARCHAR(45)
+INT INTEGER
+LIST> VARCHAR
+MAP, ?> VARCHAR
+SET> VARCHAR
+TEXT VARCHAR
+TIMESTAMP TIMESTAMP
+TIMEUUID VARCHAR
+VARCHAR VARCHAR
+VARIANT VARCHAR
+================ ======
+
+Any collection (LIST/MAP/SET) can be designated as FROZEN, and the value is
+mapped to VARCHAR. Additionally, blobs have the limitation that they cannot be empty.
+
+Types not mentioned in the table above are not supported (e.g. tuple or UDT).
+
+Partition keys can only be of the following types:
+| ASCII
+| TEXT
+| VARCHAR
+| BIGINT
+| BOOLEAN
+| DOUBLE
+| INET
+| INT
+| FLOAT
+| DECIMAL
+| TIMESTAMP
+| UUID
+| TIMEUUID
+
+Limitations
+-----------
+
+* Queries without filters containing the partition key result in fetching all partitions.
+ This causes a full scan of the entire data set, therefore it's much slower compared to a similar
+ query with a partition key as a filter.
+* ``IN`` list filters are only allowed on index (that is, partition key or clustering key) columns.
+* Range (``<`` or ``>`` and ``BETWEEN``) filters can be applied only to the partition keys.
diff --git a/presto-docs/src/main/sphinx/connector/kafka-tutorial.rst b/presto-docs/src/main/sphinx/connector/kafka-tutorial.rst
index d4d0e820e1dd7..f9dcea003b8d1 100644
--- a/presto-docs/src/main/sphinx/connector/kafka-tutorial.rst
+++ b/presto-docs/src/main/sphinx/connector/kafka-tutorial.rst
@@ -192,8 +192,8 @@ actual table shape. The raw data is available through the ``_message`` and
in JSON format, the :doc:`/functions/json` built into Presto can be used
to slice the data.
-Step 5: Add a topic decription file
------------------------------------
+Step 5: Add a topic description file
+------------------------------------
The Kafka connector supports topic description files to turn raw data into
table format. These files are located in the ``etc/kafka`` folder in the
diff --git a/presto-docs/src/main/sphinx/functions/array.rst b/presto-docs/src/main/sphinx/functions/array.rst
index f949886f77e62..0f6b3320818b6 100644
--- a/presto-docs/src/main/sphinx/functions/array.rst
+++ b/presto-docs/src/main/sphinx/functions/array.rst
@@ -101,6 +101,10 @@ Array Functions
See :func:`reduce`.
+.. function:: repeat(element, count) -> array
+
+ Repeat ``element`` for ``count`` times.
+
.. function:: reverse(x) -> array
:noindex:
diff --git a/presto-docs/src/main/sphinx/functions/binary.rst b/presto-docs/src/main/sphinx/functions/binary.rst
index cb8026e962a3e..e0cb5897f5145 100644
--- a/presto-docs/src/main/sphinx/functions/binary.rst
+++ b/presto-docs/src/main/sphinx/functions/binary.rst
@@ -42,6 +42,19 @@ Binary Functions
Decodes ``bigint`` value from a 64-bit 2's complement big endian ``binary``.
+.. function:: to_ieee754_32(real) -> varbinary
+
+ Encodes ``real`` in a 32-bit big-endian binary according to IEEE 754 single-precision floating-point format.
+
+.. function:: to_ieee754_64(double) -> varbinary
+
+ Encodes ``double`` in a 64-bit big-endian binary according to IEEE 754 double-precision floating-point format.
+
+.. function:: crc32(binary) -> bigint
+
+ Computes the CRC-32 of ``binary``. For general purpose hashing, use
+ :func:`xxhash64`, as it is much faster and produces a better quality hash.
+
.. function:: md5(binary) -> varbinary
Computes the md5 hash of ``binary``.
diff --git a/presto-docs/src/main/sphinx/functions/comparison.rst b/presto-docs/src/main/sphinx/functions/comparison.rst
index 9c3da56ed3b8a..1b6d08ce9826b 100644
--- a/presto-docs/src/main/sphinx/functions/comparison.rst
+++ b/presto-docs/src/main/sphinx/functions/comparison.rst
@@ -113,11 +113,11 @@ The following types are supported:
``TIMESTAMP WITH TIME ZONE``,
``DATE``
-.. function:: greatest(value1, value2) -> [same as input]
+.. function:: greatest(value1, value2, ..., valueN) -> [same as input]
Returns the largest of the provided values.
-.. function:: least(value1, value2) -> [same as input]
+.. function:: least(value1, value2, ..., valueN) -> [same as input]
Returns the smallest of the provided values.
@@ -153,9 +153,3 @@ Expression Meaning
==================== ===========
``ANY`` and ``SOME`` have the same meaning and can be used interchangeably.
-
-.. note::
-
- Currently, the expression ``A`` in ``A = ANY (...)`` or ``A <> ALL (...)``
- must not be ``NULL`` for any of the queried rows. Otherwise, the query will fail.
- This limitation is needed to ensure correct results and may be dropped in the future.
diff --git a/presto-docs/src/main/sphinx/functions/datetime.rst b/presto-docs/src/main/sphinx/functions/datetime.rst
index 5c758de953d70..defac4f571cf5 100644
--- a/presto-docs/src/main/sphinx/functions/datetime.rst
+++ b/presto-docs/src/main/sphinx/functions/datetime.rst
@@ -156,6 +156,32 @@ Unit Description
Returns ``timestamp2 - timestamp1`` expressed in terms of ``unit``.
+Duration Function
+-----------------
+
+The ``parse_duration`` function supports the following units:
+
+======= =============
+Unit Description
+======= =============
+``ns`` Nanoseconds
+``us`` Microseconds
+``ms`` Milliseconds
+``s`` Seconds
+``m`` Minutes
+``h`` Hours
+``d`` Days
+======= =============
+
+.. function:: parse_duration(string) -> interval
+
+ Parses ``string`` of format ``value unit`` into an interval, where
+ ``value`` is fractional number of ``unit`` values::
+
+ SELECT parse_duration('42.8ms'); -- 0 00:00:00.043
+ SELECT parse_duration('3.81 d'); -- 3 19:26:24.000
+ SELECT parse_duration('5m'); -- 0 00:05:00.000
+
MySQL Date Functions
--------------------
@@ -168,11 +194,11 @@ Specifier Description
========= ===========
``%a`` Abbreviated weekday name (``Sun`` .. ``Sat``)
``%b`` Abbreviated month name (``Jan`` .. ``Dec``)
-``%c`` Month, numeric (``0`` .. ``12``)
+``%c`` Month, numeric (``1`` .. ``12``) [#z]_
``%D`` Day of the month with English suffix (``0th``, ``1st``, ``2nd``, ``3rd``, ...)
-``%d`` Day of the month, numeric (``00`` .. ``31``)
-``%e`` Day of the month, numeric (``0`` .. ``31``)
-``%f`` Fraction of second (6 digits for printing: ``000000`` .. ``999000``; 1 - 9 digits for parsing: ``0`` .. ``999999999`` [#f]_)
+``%d`` Day of the month, numeric (``01`` .. ``31``) [#z]_
+``%e`` Day of the month, numeric (``1`` .. ``31``) [#z]_
+``%f`` Fraction of second (6 digits for printing: ``000000`` .. ``999000``; 1 - 9 digits for parsing: ``0`` .. ``999999999``) [#f]_
``%H`` Hour (``00`` .. ``23``)
``%h`` Hour (``01`` .. ``12``)
``%I`` Hour (``01`` .. ``12``)
@@ -181,7 +207,7 @@ Specifier Description
``%k`` Hour (``0`` .. ``23``)
``%l`` Hour (``1`` .. ``12``)
``%M`` Month name (``January`` .. ``December``)
-``%m`` Month, numeric (``00`` .. ``12``)
+``%m`` Month, numeric (``01`` .. ``12``) [#z]_
``%p`` ``AM`` or ``PM``
``%r`` Time, 12-hour (``hh:mm:ss`` followed by ``AM`` or ``PM``)
``%S`` Seconds (``00`` .. ``59``)
@@ -192,7 +218,7 @@ Specifier Description
``%V`` Week (``01`` .. ``53``), where Sunday is the first day of the week; used with ``%X``
``%v`` Week (``01`` .. ``53``), where Monday is the first day of the week; used with ``%x``
``%W`` Weekday name (``Sunday`` .. ``Saturday``)
-``%w`` Day of the week (``0`` .. ``6``), where Sunday is the first day of the week
+``%w`` Day of the week (``0`` .. ``6``), where Sunday is the first day of the week [#w]_
``%X`` Year for the week where Sunday is the first day of the week, numeric, four digits; used with ``%V``
``%x`` Year for the week, where Monday is the first day of the week, numeric, four digits; used with ``%v``
``%Y`` Year, numeric, four digits
@@ -203,6 +229,8 @@ Specifier Description
.. [#f] Timestamp is truncated to milliseconds.
.. [#y] When parsing, two-digit year format assumes range ``1970`` .. ``2069``, so "70" will result in year ``1970`` but "69" will produce ``2069``.
+.. [#w] This specifier is not supported yet. Consider using :func:`day_of_week` (it uses ``1-7`` instead of ``0-6``).
+.. [#z] This specifier does not support ``0`` as a month or day.
.. warning:: The following specifiers are not currently supported: ``%D %U %u %V %w %X``
diff --git a/presto-docs/src/main/sphinx/language.rst b/presto-docs/src/main/sphinx/language.rst
index 63f9d14fdeaa4..a19e4b60733b5 100644
--- a/presto-docs/src/main/sphinx/language.rst
+++ b/presto-docs/src/main/sphinx/language.rst
@@ -6,3 +6,4 @@ SQL Language
:maxdepth: 1
language/types
+ language/reserved
diff --git a/presto-docs/src/main/sphinx/language/reserved.rst b/presto-docs/src/main/sphinx/language/reserved.rst
new file mode 100644
index 0000000000000..9cf05af04d9e8
--- /dev/null
+++ b/presto-docs/src/main/sphinx/language/reserved.rst
@@ -0,0 +1,80 @@
+=================
+Reserved Keywords
+=================
+
+The following table lists all of the keywords that are reserved in Presto,
+along with their status in the SQL standard. These reserved keywords must
+be quoted (using double quotes) in order to be used as an identifier.
+
+============================== ============= =============
+Keyword SQL:2016 SQL-92
+============================== ============= =============
+``ALTER`` reserved reserved
+``AND`` reserved reserved
+``AS`` reserved reserved
+``BETWEEN`` reserved reserved
+``BY`` reserved reserved
+``CASE`` reserved reserved
+``CAST`` reserved reserved
+``CONSTRAINT`` reserved reserved
+``CREATE`` reserved reserved
+``CROSS`` reserved reserved
+``CUBE`` reserved
+``CURRENT_DATE`` reserved reserved
+``CURRENT_TIME`` reserved reserved
+``CURRENT_TIMESTAMP`` reserved reserved
+``DEALLOCATE`` reserved reserved
+``DELETE`` reserved reserved
+``DESCRIBE`` reserved reserved
+``DISTINCT`` reserved reserved
+``DROP`` reserved reserved
+``ELSE`` reserved reserved
+``END`` reserved reserved
+``ESCAPE`` reserved reserved
+``EXCEPT`` reserved reserved
+``EXECUTE`` reserved reserved
+``EXISTS`` reserved reserved
+``EXTRACT`` reserved reserved
+``FALSE`` reserved reserved
+``FOR`` reserved reserved
+``FROM`` reserved reserved
+``FULL`` reserved reserved
+``GROUP`` reserved reserved
+``GROUPING`` reserved
+``HAVING`` reserved reserved
+``IN`` reserved reserved
+``INNER`` reserved reserved
+``INSERT`` reserved reserved
+``INTERSECT`` reserved reserved
+``INTO`` reserved reserved
+``IS`` reserved reserved
+``JOIN`` reserved reserved
+``LEFT`` reserved reserved
+``LIKE`` reserved reserved
+``LOCALTIME`` reserved
+``LOCALTIMESTAMP`` reserved
+``NATURAL`` reserved reserved
+``NORMALIZE`` reserved
+``NOT`` reserved reserved
+``NULL`` reserved reserved
+``ON`` reserved reserved
+``OR`` reserved reserved
+``ORDER`` reserved reserved
+``OUTER`` reserved reserved
+``PREPARE`` reserved reserved
+``RECURSIVE`` reserved
+``RIGHT`` reserved reserved
+``ROLLUP`` reserved
+``SELECT`` reserved reserved
+``TABLE`` reserved reserved
+``THEN`` reserved reserved
+``TRUE`` reserved reserved
+``UESCAPE`` reserved
+``UNION`` reserved reserved
+``UNNEST`` reserved
+``USING`` reserved reserved
+``VALUES`` reserved reserved
+``WHEN`` reserved reserved
+``WHERE`` reserved reserved
+``WITH`` reserved reserved
+============================== ============= =============
diff --git a/presto-docs/src/main/sphinx/language/types.rst b/presto-docs/src/main/sphinx/language/types.rst
index b75fa34fe58ad..05a1bd8dac799 100644
--- a/presto-docs/src/main/sphinx/language/types.rst
+++ b/presto-docs/src/main/sphinx/language/types.rst
@@ -31,7 +31,8 @@ INTEGER
-------
A 32-bit signed two's complement integer with a minimum value of
- ``-2^31`` and a maximum value of ``2^31 - 1``.
+ ``-2^31`` and a maximum value of ``2^31 - 1``. The name INT is
+ also available for this type.
BIGINT
------
diff --git a/presto-docs/src/main/sphinx/release.rst b/presto-docs/src/main/sphinx/release.rst
index 94f0edb6e2631..2bde0e3eeab31 100644
--- a/presto-docs/src/main/sphinx/release.rst
+++ b/presto-docs/src/main/sphinx/release.rst
@@ -5,6 +5,11 @@ Release Notes
.. toctree::
:maxdepth: 1
+ release/release-0.179
+ release/release-0.178
+ release/release-0.177
+ release/release-0.176
+ release/release-0.175
release/release-0.174
release/release-0.173
release/release-0.172
diff --git a/presto-docs/src/main/sphinx/release/release-0.156.rst b/presto-docs/src/main/sphinx/release/release-0.156.rst
index a00c1325d9bb5..5e78305c50a10 100644
--- a/presto-docs/src/main/sphinx/release/release-0.156.rst
+++ b/presto-docs/src/main/sphinx/release/release-0.156.rst
@@ -2,6 +2,12 @@
Release 0.156
=============
+.. warning::
+
+ Query may incorrectly produce ``NULL`` when no row qualifies for the aggregation
+ if the ``optimize_mixed_distinct_aggregations`` session property or
+ the ``optimizer.optimize-mixed-distinct-aggregations`` config option is enabled.
+
General Changes
---------------
diff --git a/presto-docs/src/main/sphinx/release/release-0.175.rst b/presto-docs/src/main/sphinx/release/release-0.175.rst
new file mode 100644
index 0000000000000..4e249c9840718
--- /dev/null
+++ b/presto-docs/src/main/sphinx/release/release-0.175.rst
@@ -0,0 +1,36 @@
+=============
+Release 0.175
+=============
+
+General Changes
+---------------
+
+* Fix *"position is not valid"* query execution failures.
+* Fix memory accounting bug that can potentially cause ``OutOfMemoryError``.
+* Fix regression that could cause certain queries involving ``UNION`` and
+ ``GROUP BY`` or ``JOIN`` to fail during planning.
+* Fix planning failure for ``GROUP BY`` queries containing correlated
+ subqueries in the ``SELECT`` clause.
+* Fix execution failure for certain ``DELETE`` queries.
+* Reduce occurrences of *"Method code too large"* errors.
+* Reduce memory utilization for certain queries involving ``ORDER BY``.
+* Improve performance of map subscript from O(n) to O(1) when the map is
+ produced by an eligible operation, including the map constructor and
+ Hive readers (except ORC and optimized Parquet). More read and write
+ operations will take advantage of this in future releases.
+* Add ``enable_intermediate_aggregations`` session property to enable the
+ use of intermediate aggregations within un-grouped aggregations.
+* Add support for ``INTERVAL`` data type to :func:`avg` and :func:`sum` aggregation functions.
+* Add support for ``INT`` as an alias for the ``INTEGER`` data type.
+* Add resource group information to query events.
+
+Hive Changes
+------------
+
+* Make table creation metastore operations idempotent, which allows
+ recovery when retrying timeouts or other errors.
+
+MongoDB Changes
+---------------
+
+* Rename ``mongodb.connection-per-host`` config option to ``mongodb.connections-per-host``.
diff --git a/presto-docs/src/main/sphinx/release/release-0.176.rst b/presto-docs/src/main/sphinx/release/release-0.176.rst
new file mode 100644
index 0000000000000..369ca619bb255
--- /dev/null
+++ b/presto-docs/src/main/sphinx/release/release-0.176.rst
@@ -0,0 +1,29 @@
+=============
+Release 0.176
+=============
+
+General Changes
+---------------
+
+* Fix an issue where a query (and some of its tasks) continues to
+ consume CPU/memory on the coordinator and workers after the query fails.
+* Fix a regression that cause the GC overhead and pauses to increase significantly when processing maps.
+* Fix a memory tracking bug that causes the memory to be overestimated for ``GROUP BY`` queries on ``bigint`` columns.
+* Improve the performance of the :func:`transform_values` function.
+* Add support for casting from ``JSON`` to ``REAL`` type.
+* Add :func:`parse_duration` function.
+
+MySQL Changes
+-------------
+
+* Disallow having a database in the ``connection-url`` config property.
+
+Accumulo Changes
+----------------
+
+* Decrease planning time by fetching index metrics in parallel.
+
+MongoDB Changes
+---------------
+
+* Allow predicate pushdown for ObjectID.
diff --git a/presto-docs/src/main/sphinx/release/release-0.177.rst b/presto-docs/src/main/sphinx/release/release-0.177.rst
new file mode 100644
index 0000000000000..0f5081bb313f9
--- /dev/null
+++ b/presto-docs/src/main/sphinx/release/release-0.177.rst
@@ -0,0 +1,77 @@
+=============
+Release 0.177
+=============
+
+.. warning::
+
+ Query may incorrectly produce ``NULL`` when no row qualifies for the aggregation
+ if the ``optimize_mixed_distinct_aggregations`` session property or
+ the ``optimizer.optimize-mixed-distinct-aggregations`` config option is enabled.
+ This optimization was introduced in Presto version 0.156.
+
+General Changes
+---------------
+
+* Fix correctness issue when performing range comparisons over columns of type ``CHAR``.
+* Fix correctness issue due to mishandling of nulls and non-deterministic expressions in
+ inequality joins unless ``fast_inequality_join`` is disabled.
+* Fix excessive GC overhead caused by lambda expressions. There are still known GC issues
+ with captured lambda expressions. This will be fixed in a future release.
+* Check for duplicate columns in ``CREATE TABLE`` before asking the connector to create
+ the table. This improves the error message for most connectors and will prevent errors
+ for connectors that do not perform validation internally.
+* Add support for null values on the left-hand side of a semijoin (i.e., ``IN`` predicate
+ with subqueries).
+* Add ``SHOW STATS`` to display table and query statistics.
+* Improve implicit coercion support for functions involving lambda. Specifically, this makes
+ it easier to use the :func:`reduce` function.
+* Improve plans for queries involving ``ORDER BY`` and ``LIMIT`` by avoiding unnecessary
+ data exchanges.
+* Improve performance of queries containing window functions with identical ``PARTITION BY``
+ and ``ORDER BY`` clauses.
+* Improve performance of certain queries involving ``OUTER JOIN`` and aggregations, or
+ containing certain forms of correlated subqueries. This optimization is experimental
+ and can be turned on via the ``push_aggregation_through_join`` session property or the
+ ``optimizer.push-aggregation-through-join`` config option.
+* Improve performance of certain queries involving joins and aggregations. This optimization
+ is experimental and can be turned on via the ``push_partial_aggregation_through_join``
+ session property.
+* Improve error message when a lambda expression has a different number of arguments than expected.
+* Improve error message when certain invalid ``GROUP BY`` expressions containing lambda expressions.
+
+Hive Changes
+------------
+
+* Fix handling of trailing spaces for the ``CHAR`` type when reading RCFile.
+* Allow inserts into tables that have more partitions than the partitions-per-scan limit.
+* Add support for exposing Hive table statistics to the engine. This option is experimental and
+ can be turned on via the ``statistics_enabled`` session property.
+* Ensure file name is always present for error messages about corrupt ORC files.
+
+Cassandra Changes
+-----------------
+
+* Remove caching of metadata in the Cassandra connector. Metadata caching makes Presto violate
+ the consistency defined by the Cassandra cluster. It's also unnecessary because the Cassandra
+ driver internally caches metadata. The ``cassandra.max-schema-refresh-threads``,
+ ``cassandra.schema-cache-ttl`` and ``cassandra.schema-refresh-interval`` config options have
+ been removed.
+* Fix intermittent issue in the connection retry mechanism.
+
+Web UI Changes
+--------------
+
+* Change cluster HUD realtime statistics to be aggregated across all running queries.
+* Change parallelism statistic on cluster HUD to be averaged per-worker.
+* Fix bug that always showed indeterminate progress bar in query list view.
+* Change running drivers statistic to exclude blocked drivers.
+* Change unit of CPU and scheduled time rate sparklines to seconds on query details page.
+* Change query details page refresh interval to three seconds.
+* Add uptime and connected status indicators to every page.
+
+CLI Changes
+-----------
+
+* Add support for preprocessing commands. When the ``PRESTO_PREPROCESSOR`` environment
+ variable is set, all commands are piped through the specified program before being sent to
+ the Presto server.
diff --git a/presto-docs/src/main/sphinx/release/release-0.178.rst b/presto-docs/src/main/sphinx/release/release-0.178.rst
new file mode 100644
index 0000000000000..2912a9775fcab
--- /dev/null
+++ b/presto-docs/src/main/sphinx/release/release-0.178.rst
@@ -0,0 +1,31 @@
+=============
+Release 0.178
+=============
+
+General Changes
+---------------
+
+* Fix various memory accounting bugs, which reduces the likelihood of full GCs/OOMs.
+* Fix a regression that causes queries that use the keyword "stats" to fail to parse.
+* Fix an issue where a query does not get cleaned up on the coordinator after query failure.
+* Add ability to cast to ``JSON`` from ``REAL``, ``TINYINT`` or ``SMALLINT``.
+* Add support for ``GROUPING`` operation to :ref:`complex grouping operations`.
+* Add support for correlated subqueries in ``IN`` predicates.
+* Add :func:`to_ieee754_32` and :func:`to_ieee754_64` functions.
+
+Hive Changes
+------------
+
+* Fix high CPU usage due to schema caching when reading Avro files.
+* Preserve decompression error causes when decoding ORC files.
+
+Memory Connector Changes
+------------------------
+
+* Fix a bug that prevented creating empty tables.
+
+SPI Changes
+-----------
+
+* Make environment available to resource group configuration managers.
+* Add additional performance statistics to query completion event.
diff --git a/presto-docs/src/main/sphinx/release/release-0.179.rst b/presto-docs/src/main/sphinx/release/release-0.179.rst
new file mode 100644
index 0000000000000..727faa9455637
--- /dev/null
+++ b/presto-docs/src/main/sphinx/release/release-0.179.rst
@@ -0,0 +1,40 @@
+=============
+Release 0.179
+=============
+
+General Changes
+---------------
+
+* Fix issue which could cause incorrect results when processing dictionary encoded data. If the expression
+ can fail on bad input, the results from filtered-out rows containing bad input may be included in the query
+ output. See `#8262 `_ for more details.
+* Fix planning failure when similar expressions appear in the ``ORDER BY`` clause of a query that
+ contains ``ORDER BY`` and ``LIMIT``.
+* Fix planning failure when ``GROUPING()`` is used with the ``legacy_order_by`` session property set to ``true``.
+* Fix parsing failure when ``NFD``, ``NFC``, ``NFKD`` or ``NFKC`` are used as identifiers.
+* Fix a memory leak on the coordinator that manifests itself with canceled queries.
+* Fix excessive GC overhead caused by captured lambda expressions.
+* Reduce the memory usage of map/array aggregation functions.
+* Redact sensitive config property values in the server log.
+* Update timezone database to version 2017b.
+* Add :func:`repeat` function.
+* Add :func:`crc32` function.
+* Add file based global security, which can be configured with the ``etc/access-control.properties``
+ and ``security.config-file`` config properties. See :doc:`/security/built-in-system-access-control`
+ for more details.
+* Add support for configuring query runtime and queueing time limits to resource groups.
+
+Hive Changes
+------------
+
+* Fail queries that access encrypted S3 objects that do not have their unencrypted content lengths set in their metadata.
+
+JDBC Driver Changes
+-------------------
+
+* Add support for setting query timeout through ``Statement.setQueryTimeout()``.
+
+SPI Changes
+-----------
+
+* Add grantee and revokee to ``GRANT`` and ``REVOKE`` security checks.
diff --git a/presto-docs/src/main/sphinx/rest.rst b/presto-docs/src/main/sphinx/rest.rst
index d65b9927a7595..3deb9709d31b8 100644
--- a/presto-docs/src/main/sphinx/rest.rst
+++ b/presto-docs/src/main/sphinx/rest.rst
@@ -13,7 +13,6 @@ responses.
.. toctree::
:maxdepth: 1
- rest/execute
rest/node
rest/query
rest/stage
@@ -30,12 +29,6 @@ The Presto REST API contains several, high-level resources that
correspond to the components of a Presto installation.
-Execute Resource
-
- The execute resource is what the client sends queries to. It is
- available at the path ``/v1/execute``, and accepts a query as a POST
- and returns JSON.
-
Query Resource
The query resource takes a SQL query. It is available at the path
diff --git a/presto-docs/src/main/sphinx/rest/execute.rst b/presto-docs/src/main/sphinx/rest/execute.rst
deleted file mode 100644
index 098d4de1d2972..0000000000000
--- a/presto-docs/src/main/sphinx/rest/execute.rst
+++ /dev/null
@@ -1,63 +0,0 @@
-================
-Execute Resource
-================
-
-.. function:: POST /v1/execute
-
- :Body: SQL Query to execute
- :Header "X-Presto-User": User to execute statement on behalf of (optional)
- :Header "X-Presto-Source": Source of query
- :Header "X-Presto-Catalog": Catalog to execute query against
- :Header "X-Presto-Schema": Schema to execute query against
-
- Call this to execute a SQL statement as an alternative to running
- ``/v1/statement``. Where ``/v1/statement`` will return a
- ``nextUri`` and details about a running query, the ``/v1/execute``
- call will simply execute the SQL statement posted to it and return
- the result set. This service will not return updates about query
- status or details about stages and tasks. It simply executes a
- query and returns the result.
-
- The sample request and response shown below demonstrate how the
- execute call works. Once you post a SQL statement to /v1/execute it
- returns a set of columns describing an array of data items. This
- trivial executes a "show functions" statement.
-
- **Example request**:
-
- .. sourcecode:: http
-
- POST /v1/execute HTTP/1.1
- Host: localhost:8001
- X-Presto-Schema: jmx
- X-Presto-User: tobrie1
- X-Presto-Catalog: jmx
- Content-Type: text/html
- Content-Length: 14
-
- show functions
-
- **Example response**:
-
- .. sourcecode:: http
-
- HTTP/1.1 200 OK
- Content-Type: application/json
- X-Content-Type-Options: nosniff
- Transfer-Encoding: chunked
-
- {"columns":
- [
- {"name":"Function","type":"varchar"},
- {"name":"Return Type","type":"varchar"},
- {"name":"Argument Types","type":"varchar"},
- {"name":"Function Type","type":"varchar"},
- {"name":"Description","type":"varchar"}
- ],
- "data":
- [
- ["abs","bigint","bigint","scalar","absolute value"],
- ["abs","double","double","scalar","absolute value"],
- ...
- ]
- };
diff --git a/presto-docs/src/main/sphinx/security.rst b/presto-docs/src/main/sphinx/security.rst
index 921bd18f09542..27b08492d4d06 100644
--- a/presto-docs/src/main/sphinx/security.rst
+++ b/presto-docs/src/main/sphinx/security.rst
@@ -9,3 +9,5 @@ Security
security/cli
security/ldap
security/tls
+ security/built-in-system-access-control
+ security/internal-communication
diff --git a/presto-docs/src/main/sphinx/security/built-in-system-access-control.rst b/presto-docs/src/main/sphinx/security/built-in-system-access-control.rst
new file mode 100644
index 0000000000000..5012d8fc5cde1
--- /dev/null
+++ b/presto-docs/src/main/sphinx/security/built-in-system-access-control.rst
@@ -0,0 +1,111 @@
+==============================
+Built-in System Access Control
+==============================
+
+A system access control plugin enforces authorization at a global level,
+before any connector level authorization. You can either use one of the built-in
+plugins in Presto or provide your own by following the guidelines in
+:doc:`/develop/system-access-control`. Presto offers three built-in plugins:
+
+================================================== ============================================================
+Plugin Name Description
+================================================== ============================================================
+``allow-all`` (default value) All operations are permitted.
+
+``read-only`` Operations that read data or metadata are permitted, but
+ none of the operations that write data or metadata are
+ allowed. See :ref:`read-only-system-access-control` for
+ details.
+
+``file`` Authorization checks are enforced using a config file
+ specified by the configuration property ``security.config-file``.
+ See :ref:`file-based-system-access-control` for details.
+================================================== ============================================================
+
+Allow All System Access Control
+===============================
+
+All operations are permitted under this plugin. This plugin is enabled by default.
+
+.. _read-only-system-access-control:
+
+Read Only System Access Control
+===============================
+
+Under this plugin, you are allowed to execute any operation that reads data or
+metadata, such as ``SELECT`` or ``SHOW``. Setting system level or catalog level
+session properties is also permitted. However, any operation that writes data or
+metadata, such as ``CREATE``, ``INSERT`` or ``DELETE``, is prohibited.
+To use this plugin, add an ``etc/access-control.properties``
+file with the following contents:
+
+.. code-block:: none
+
+ access-control.name=read-only
+
+.. _file-based-system-access-control:
+
+File Based System Access Control
+================================
+
+This plugin allows you to specify access control rules in a file. To use this
+plugin, add an ``etc/access-control.properties`` file containing two required
+properties: ``access-control.name``, which must be equal to ``file``, and
+``security.config-file``, which must be equal to the location of the config file.
+For example, if a config file named ``rules.json``
+resides in ``etc``, add an ``etc/access-control.properties`` with the following
+contents:
+
+.. code-block:: none
+
+ access-control.name=file
+ security.config-file=etc/rules.json
+
+The config file consists of a list of access control rules in JSON format. The
+rules are matched in the order specified in the file. All
+regular expressions default to ``.*`` if not specified.
+
+This plugin currently only supports catalog access control rules. If you want
+to limit access on a system level in any other way, you must implement a custom
+SystemAccessControl plugin (see :doc:`/develop/system-access-control`).
+
+Catalog Rules
+-------------
+
+These rules govern the catalogs particular users can access. The user is
+granted access to a catalog based on the first matching rule. If no rule
+matches, access is denied. Each rule is composed of the following fields:
+
+* ``user`` (optional): regex to match against user name.
+* ``catalog`` (optional): regex to match against catalog name.
+* ``allowed`` (required): boolean indicating whether a user has access to the catalog
+
+.. note::
+
+ By default, all users have access to the ``system`` catalog. You can
+ override this behavior by adding a rule.
+
+For example, if you want to allow only the user ``admin`` to access the
+``mysql`` and the ``system`` catalog, allow all users to access the ``hive``
+catalog, and deny all other access, you can use the following rules:
+
+.. code-block:: json
+
+ {
+ "catalogs": [
+ {
+ "user": "admin",
+ "catalog": "(mysql|system)",
+ "allow": true
+ },
+ {
+ "catalog": "hive",
+ "allow": true
+ },
+ {
+ "catalog": "system",
+ "allow": false
+ }
+ ]
+ }
+
diff --git a/presto-docs/src/main/sphinx/security/internal-communication.rst b/presto-docs/src/main/sphinx/security/internal-communication.rst
new file mode 100644
index 0000000000000..4a1092a8fc84d
--- /dev/null
+++ b/presto-docs/src/main/sphinx/security/internal-communication.rst
@@ -0,0 +1,156 @@
+=============================
+Secure Internal Communication
+=============================
+
+The Presto cluster can be configured to use secured communication. Communication
+between Presto nodes can be secured with SSL/TLS.
+
+Internal SSL/TLS configuration
+------------------------------
+
+SSL/TLS is configured in the `config.properties` file. The SSL/TLS on the
+worker and coordinator nodes are configured using the same set of properties.
+Every node in the cluster must be configured. Nodes that have not been
+configured, or are configured incorrectly, will not be able to communicate with
+other nodes in the cluster.
+
+To enable SSL/TLS for Presto internal communication, do the following:
+
+1. Disable HTTP endpoint.
+
+ .. code-block:: none
+
+ http-server.http.enabled=false
+
+ .. warning::
+
+ You can enable HTTPS while leaving HTTP enabled. In most cases this is a
+ security hole. If you are certain you want to use this configuration, you
+ should consider using an firewall to limit access to the HTTP endpoint to
+ only those hosts that should be allowed to use it.
+
+2. Configure the cluster to communicate using the fully qualified domain name (fqdn)
+ of the cluster nodes. This can be done in either of the following ways:
+
+ - If the DNS service is configured properly, we can just let the nodes to
+ introduce themselves to the coordinator using the hostname taken from
+ the system configuration (`hostname --fqdn`)
+
+ .. code-block:: none
+
+ node.internal-address-source=FQDN
+
+ - It is also possible to specify each node's fully-qualified hostname manually.
+ This will be different for every host. Hosts should be in the same domain to
+ make it easy to create the correct SSL/TLS certificates.
+ e.g.: `coordinator.example.com`, `worker1.example.com`, `worker2.example.com`.
+
+ .. code-block:: none
+
+ node.internal-address=
+
+
+3. Generate a Java Keystore File. Every Presto node must be able to connect to
+ any other node within the same cluster. It is possible to create unique
+ certificates for every node using the fully-qualified hostname of each host,
+ create a keystore that contains all the public keys for all of the hosts,
+ and specify it for the client (`http-client.https.keystore.path`). In most
+ cases it will be simpler to use a wildcard in the certificate as shown
+ below.
+
+ .. code-block:: none
+
+ keytool -genkeypair -alias example.com -keyalg RSA -keystore keystore.jks
+ Enter keystore password:
+ Re-enter new password:
+ What is your first and last name?
+ [Unknown]: *.example.com
+ What is the name of your organizational unit?
+ [Unknown]:
+ What is the name of your organization?
+ [Unknown]:
+ What is the name of your City or Locality?
+ [Unknown]:
+ What is the name of your State or Province?
+ [Unknown]:
+ What is the two-letter country code for this unit?
+ [Unknown]:
+ Is CN=*.example.com, OU=Unknown, O=Unknown, L=Unknown, ST=Unknown, C=Unknown correct?
+ [no]: yes
+
+ Enter key password for
+ (RETURN if same as keystore password):
+
+ .. Note: Replace `example.com` with the appropriate domain.
+
+4. Distribute the Java Keystore File across the Presto cluster.
+
+5. Enable the HTTPS endpoint.
+
+ .. code-block:: none
+
+ http-server.https.enabled=true
+ http-server.https.port=
+ http-server.https.keystore.path=
+ http-server.https.keystore.key=
+
+6. Change the discovery uri to HTTPS.
+
+ .. code-block:: none
+
+ discovery.uri=https://:
+
+7. Configure the internal communication to require HTTPS.
+
+ .. code-block:: none
+
+ internal-communication.https.required=true
+
+8. Configure the internal communication to use the Java keystore file.
+
+ .. code-block:: none
+
+ internal-communication.https.keystore.path=
+ internal-communication.https.keystore.key=
+
+
+Performance with SSL/TLS enabled
+--------------------------------
+
+Enabling encryption impacts performance. The performance degradation can vary
+based on the environment, queries, and concurrency.
+
+For queries that do not require transferring too much data between the Presto
+nodes (e.g. `SELECT count(*) FROM table`), the performance impact is negligible.
+
+However, for CPU intensive queries which require a considerable amount of data
+to be transferred between the nodes (for example, distributed joins, aggregations and
+window functions, which require repartitioning), the performance impact might be
+considerable. The slowdown may vary from 10% to even 100%+, depending on the network
+traffic and the CPU utilization.
+
+Advanced Performance Tuning
+---------------------------
+
+In some cases, changing the source of random numbers will improve performance
+significantly.
+
+By default, TLS encryption uses the `/dev/urandom` system device as a source of entropy.
+This device has limited throughput, so on environments with high network bandwidth
+(e.g. InfiniBand), it may become a bottleneck. In such situations, it is recommended to try
+to switch the random number generator algorithm to `SHA1PRNG`, by setting it via
+`http-server.https.secure-random-algorithm` property in `config.properties` on the coordinator
+and all of the workers:
+
+ .. code-block:: none
+
+ http-server.https.secure-random-algorithm=SHA1PRNG
+
+Be aware that this algorithm takes the initial seed from
+the blocking `/dev/random` device. For environments that do not have enough entropy to seed
+the `SHAPRNG` algorithm, the source can be changed to `/dev/urandom`
+by adding the `java.security.egd` property to `jvm.config`:
+
+ .. code-block:: none
+
+ -Djava.security.egd=file:/dev/urandom
diff --git a/presto-docs/src/main/sphinx/sql/select.rst b/presto-docs/src/main/sphinx/sql/select.rst
index 07a9bb3682b1d..441476f042484 100644
--- a/presto-docs/src/main/sphinx/sql/select.rst
+++ b/presto-docs/src/main/sphinx/sql/select.rst
@@ -364,6 +364,48 @@ only unique grouping sets are generated::
The default set quantifier is ``ALL``.
+**GROUPING Operation**
+
+``grouping(col1, ..., colN) -> bigint``
+
+The grouping operation returns a bit set converted to decimal, indicating which columns are present in a
+grouping. It must be used in conjunction with ``GROUPING SETS``, ``ROLLUP``, ``CUBE`` or ``GROUP BY``
+and its arguments must match exactly the columns referenced in the corresponding ``GROUPING SETS``,
+``ROLLUP``, ``CUBE`` or ``GROUP BY`` clause.
+
+To compute the resulting bit set for a particular row, bits are assigned to the argument columns with
+the rightmost column being the least significant bit. For a given grouping, a bit is set to 0 if the
+corresponding column is included in the grouping and to 1 otherwise. For example, consider the query
+below::
+
+ SELECT origin_state, origin_zip, destination_state, sum(package_weight),
+ grouping(origin_state, origin_zip, destination_state)
+ FROM shipping
+ GROUP BY GROUPING SETS (
+ (origin_state),
+ (origin_state, origin_zip),
+ (destination_state));
+
+.. code-block:: none
+
+ origin_state | origin_zip | destination_state | _col3 | _col4
+ --------------+------------+-------------------+-------+-------
+ California | NULL | NULL | 1397 | 3
+ New Jersey | NULL | NULL | 225 | 3
+ New York | NULL | NULL | 3 | 3
+ California | 94131 | NULL | 60 | 1
+ New Jersey | 7081 | NULL | 225 | 1
+ California | 90210 | NULL | 1337 | 1
+ New York | 10002 | NULL | 3 | 1
+ NULL | NULL | New Jersey | 58 | 6
+ NULL | NULL | Connecticut | 1562 | 6
+ NULL | NULL | Colorado | 5 | 6
+ (10 rows)
+
+The first grouping in the above result only includes the ``origin_state`` column and excludes
+the ``origin_zip`` and ``destination_state`` columns. The bit set constructed for that grouping
+is ``011`` where the most significant bit represents ``origin_state``.
+
HAVING Clause
-------------
@@ -763,12 +805,6 @@ standard rules for nulls. The subquery must produce exactly one column::
FROM nation
WHERE regionkey IN (SELECT regionkey FROM region)
-.. note::
-
- Currently, the expression on the left hand side of ``IN`` must
- not be ``NULL`` for any of the queried rows. Otherwise, the query will fail.
- This limitation is needed to ensure correct results and may be dropped in the future.
-
Scalar Subquery
^^^^^^^^^^^^^^^
diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml
index be5d3b86e5b46..0ae0b5e9b3c31 100644
--- a/presto-example-http/pom.xml
+++ b/presto-example-http/pom.xml
@@ -4,7 +4,7 @@
com.facebook.presto
presto-root
- 0.175-SNAPSHOT
+ 0.180-SNAPSHOT
presto-example-http
diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml
deleted file mode 100644
index 3727b81968124..0000000000000
--- a/presto-hive-cdh4/pom.xml
+++ /dev/null
@@ -1,122 +0,0 @@
-
-
- 4.0.0
-
-
- com.facebook.presto
- presto-root
- 0.175-SNAPSHOT
-
-
- presto-hive-cdh4
- Presto - Hive Connector - CDH 4
- presto-plugin
-
-
- ${project.parent.basedir}
-
-
-
-
- com.facebook.presto
- presto-hive
-
-
-
- com.facebook.presto.hadoop
- hadoop-cdh4
- runtime
-
-
-
-
- com.facebook.presto
- presto-spi
- provided
-
-
-
- io.airlift
- slice
- provided
-
-
-
- io.airlift
- units
- provided
-
-
-
- com.fasterxml.jackson.core
- jackson-annotations
- provided
-
-
-
- org.openjdk.jol
- jol-core
- provided
-
-
-
-
- org.testng
- testng
- test
-
-
-
- io.airlift
- testing
- test
-
-
-
- com.facebook.presto
- presto-hive
- test-jar
- test
-
-
-
- com.facebook.presto
- presto-main
- test
-
-
-
-
-
-
- org.apache.maven.plugins
- maven-surefire-plugin
-
- hive,hive-s3
-
-
-
-
-
-
-
- test-hive-cdh4
-
-
-
- org.apache.maven.plugins
- maven-surefire-plugin
-
- hive-s3
-
- localhost
- 9083
- default
-
-
-
-
-
-
-
-
diff --git a/presto-hive-cdh4/src/test/java/com/facebook/presto/hive/TestHiveClientS3.java b/presto-hive-cdh4/src/test/java/com/facebook/presto/hive/TestHiveClientS3.java
deleted file mode 100644
index d983920a1ad03..0000000000000
--- a/presto-hive-cdh4/src/test/java/com/facebook/presto/hive/TestHiveClientS3.java
+++ /dev/null
@@ -1,38 +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.hive;
-
-import org.testng.annotations.BeforeClass;
-import org.testng.annotations.Parameters;
-import org.testng.annotations.Test;
-
-@Test(groups = "hive-s3")
-public class TestHiveClientS3
- extends AbstractTestHiveClientS3
-{
- @Parameters({
- "hive.cdh4.metastoreHost",
- "hive.cdh4.metastorePort",
- "hive.cdh4.databaseName",
- "hive.cdh4.s3.awsAccessKey",
- "hive.cdh4.s3.awsSecretKey",
- "hive.cdh4.s3.writableBucket",
- })
- @BeforeClass
- @Override
- public void setup(String host, int port, String databaseName, String awsAccessKey, String awsSecretKey, String writableBucket)
- {
- super.setup(host, port, databaseName, awsAccessKey, awsSecretKey, writableBucket);
- }
-}
diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml
deleted file mode 100644
index d3f28efd2d987..0000000000000
--- a/presto-hive-cdh5/pom.xml
+++ /dev/null
@@ -1,122 +0,0 @@
-
-
- 4.0.0
-
-
- com.facebook.presto
- presto-root
- 0.175-SNAPSHOT
-
-
- presto-hive-cdh5
- Presto - Hive Connector - CDH 5
- presto-plugin
-
-
- ${project.parent.basedir}
-
-
-
-
- com.facebook.presto
- presto-hive
-
-
-
- com.facebook.presto.hadoop
- hadoop-apache2
- runtime
-
-
-
-
- com.facebook.presto
- presto-spi
- provided
-
-
-
- io.airlift
- slice
- provided
-
-
-
- io.airlift
- units
- provided
-
-
-
- com.fasterxml.jackson.core
- jackson-annotations
- provided
-
-
-
- org.openjdk.jol
- jol-core
- provided
-
-
-
-
- org.testng
- testng
- test
-
-
-
- io.airlift
- testing
- test
-
-
-
- com.facebook.presto
- presto-hive
- test-jar
- test
-
-
-
- com.facebook.presto
- presto-main
- test
-
-
-
-
-
-
- org.apache.maven.plugins
- maven-surefire-plugin
-
- hive,hive-s3
-
-
-
-
-
-
-
- test-hive-cdh5
-
-
-
- org.apache.maven.plugins
- maven-surefire-plugin
-
- hive-s3
-
- localhost
- 9083
- default
-
-
-
-
-
-
-
-
diff --git a/presto-hive-cdh5/src/test/java/com/facebook/presto/hive/TestHiveClientS3.java b/presto-hive-cdh5/src/test/java/com/facebook/presto/hive/TestHiveClientS3.java
deleted file mode 100644
index bbc906b4d6153..0000000000000
--- a/presto-hive-cdh5/src/test/java/com/facebook/presto/hive/TestHiveClientS3.java
+++ /dev/null
@@ -1,38 +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.hive;
-
-import org.testng.annotations.BeforeClass;
-import org.testng.annotations.Parameters;
-import org.testng.annotations.Test;
-
-@Test(groups = "hive-s3")
-public class TestHiveClientS3
- extends AbstractTestHiveClientS3
-{
- @Parameters({
- "hive.cdh5.metastoreHost",
- "hive.cdh5.metastorePort",
- "hive.cdh5.databaseName",
- "hive.cdh5.s3.awsAccessKey",
- "hive.cdh5.s3.awsSecretKey",
- "hive.cdh5.s3.writableBucket",
- })
- @BeforeClass
- @Override
- public void setup(String host, int port, String databaseName, String awsAccessKey, String awsSecretKey, String writableBucket)
- {
- super.setup(host, port, databaseName, awsAccessKey, awsSecretKey, writableBucket);
- }
-}
diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml
deleted file mode 100644
index 1143e04717a39..0000000000000
--- a/presto-hive-hadoop1/pom.xml
+++ /dev/null
@@ -1,122 +0,0 @@
-
-
- 4.0.0
-
-
- com.facebook.presto
- presto-root
- 0.175-SNAPSHOT
-
-
- presto-hive-hadoop1
- Presto - Hive Connector - Apache Hadoop 1.x
- presto-plugin
-
-
- ${project.parent.basedir}
-
-
-
-
- com.facebook.presto
- presto-hive
-
-
-
- com.facebook.presto.hadoop
- hadoop-apache1
- runtime
-
-
-
-
- com.facebook.presto
- presto-spi
- provided
-
-
-
- io.airlift
- slice
- provided
-
-
-
- io.airlift
- units
- provided
-
-
-
- com.fasterxml.jackson.core
- jackson-annotations
- provided
-
-
-
- org.openjdk.jol
- jol-core
- provided
-
-
-
-
- org.testng
- testng
- test
-
-
-
- io.airlift
- testing
- test
-
-
-
- com.facebook.presto
- presto-hive
- test-jar
- test
-
-
-
- com.facebook.presto
- presto-main
- test
-
-
-
-
-
-
- org.apache.maven.plugins
- maven-surefire-plugin
-
- hive,hive-s3
-
-
-
-
-
-
-
- test-hive-hadoop1
-
-
-
- org.apache.maven.plugins
- maven-surefire-plugin
-
- hive-s3
-
- localhost
- 9083
- default
-
-
-
-
-
-
-
-
diff --git a/presto-hive-hadoop1/src/test/java/com/facebook/presto/hive/TestHiveClientS3.java b/presto-hive-hadoop1/src/test/java/com/facebook/presto/hive/TestHiveClientS3.java
deleted file mode 100644
index cbe5778a596fe..0000000000000
--- a/presto-hive-hadoop1/src/test/java/com/facebook/presto/hive/TestHiveClientS3.java
+++ /dev/null
@@ -1,38 +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.hive;
-
-import org.testng.annotations.BeforeClass;
-import org.testng.annotations.Parameters;
-import org.testng.annotations.Test;
-
-@Test(groups = "hive-s3")
-public class TestHiveClientS3
- extends AbstractTestHiveClientS3
-{
- @Parameters({
- "hive.hadoop1.metastoreHost",
- "hive.hadoop1.metastorePort",
- "hive.hadoop1.databaseName",
- "hive.hadoop1.s3.awsAccessKey",
- "hive.hadoop1.s3.awsSecretKey",
- "hive.hadoop1.s3.writableBucket",
- })
- @BeforeClass
- @Override
- public void setup(String host, int port, String databaseName, String awsAccessKey, String awsSecretKey, String writableBucket)
- {
- super.setup(host, port, databaseName, awsAccessKey, awsSecretKey, writableBucket);
- }
-}
diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml
index adbabf068fe06..9db506f5d5419 100644
--- a/presto-hive-hadoop2/pom.xml
+++ b/presto-hive-hadoop2/pom.xml
@@ -5,7 +5,7 @@
com.facebook.presto
presto-root
- 0.175-SNAPSHOT
+ 0.180-SNAPSHOT
presto-hive-hadoop2
diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml
index 49b8e3de1e8ba..322bdb9476f81 100644
--- a/presto-hive/pom.xml
+++ b/presto-hive/pom.xml
@@ -5,7 +5,7 @@
com.facebook.presto
presto-root
- 0.175-SNAPSHOT
+ 0.180-SNAPSHOT
presto-hive
@@ -39,7 +39,7 @@
com.facebook.presto.hadoop
- hadoop-cdh4
+ hadoop-apache2
provided
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/BackgroundHiveSplitLoader.java b/presto-hive/src/main/java/com/facebook/presto/hive/BackgroundHiveSplitLoader.java
index f0c6b3d50c68a..0091e26c05ba6 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/BackgroundHiveSplitLoader.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/BackgroundHiveSplitLoader.java
@@ -77,6 +77,7 @@
import static com.facebook.presto.hive.HiveUtil.getInputFormat;
import static com.facebook.presto.hive.HiveUtil.isSplittable;
import static com.facebook.presto.hive.metastore.MetastoreUtil.getHiveSchema;
+import static com.facebook.presto.hive.util.ConfigurationUtils.toJobConf;
import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED;
import static com.google.common.base.Preconditions.checkState;
import static java.lang.Math.toIntExact;
@@ -301,7 +302,7 @@ private void loadPartition(HivePartitionMetadata partition)
TextInputFormat targetInputFormat = new TextInputFormat();
// get the configuration for the target path -- it may be a different hdfs instance
Configuration targetConfiguration = hdfsEnvironment.getConfiguration(targetPath);
- JobConf targetJob = new JobConf(targetConfiguration);
+ JobConf targetJob = toJobConf(targetConfiguration);
targetJob.setInputFormat(TextInputFormat.class);
targetInputFormat.configure(targetJob);
FileInputFormat.setInputPaths(targetJob, targetPath);
@@ -317,7 +318,7 @@ private void loadPartition(HivePartitionMetadata partition)
// To support custom input formats, we want to call getSplits()
// on the input format to obtain file splits.
if (shouldUseFileSplitsFromInputFormat(inputFormat)) {
- JobConf jobConf = new JobConf(configuration);
+ JobConf jobConf = toJobConf(configuration);
FileInputFormat.setInputPaths(jobConf, path);
InputSplit[] splits = inputFormat.getSplits(jobConf, 0);
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/ColumnarBinaryHiveRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/hive/ColumnarBinaryHiveRecordCursor.java
deleted file mode 100644
index 5ec4a67985abb..0000000000000
--- a/presto-hive/src/main/java/com/facebook/presto/hive/ColumnarBinaryHiveRecordCursor.java
+++ /dev/null
@@ -1,755 +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.hive;
-
-import com.facebook.presto.spi.PrestoException;
-import com.facebook.presto.spi.RecordCursor;
-import com.facebook.presto.spi.type.DecimalType;
-import com.facebook.presto.spi.type.Type;
-import com.facebook.presto.spi.type.TypeManager;
-import com.google.common.base.Throwables;
-import com.google.common.collect.ImmutableSet;
-import io.airlift.slice.ByteArrays;
-import io.airlift.slice.Slice;
-import io.airlift.slice.Slices;
-import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable;
-import org.apache.hadoop.hive.serde2.columnar.BytesRefWritable;
-import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
-import org.apache.hadoop.hive.serde2.lazy.ByteArrayRef;
-import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryFactory;
-import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryObject;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
-import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
-import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
-import org.apache.hadoop.io.WritableUtils;
-import org.apache.hadoop.mapred.RecordReader;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Properties;
-import java.util.Set;
-
-import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR;
-import static com.facebook.presto.hive.HiveErrorCode.HIVE_BAD_DATA;
-import static com.facebook.presto.hive.HiveErrorCode.HIVE_CURSOR_ERROR;
-import static com.facebook.presto.hive.HiveType.HIVE_BYTE;
-import static com.facebook.presto.hive.HiveType.HIVE_DATE;
-import static com.facebook.presto.hive.HiveType.HIVE_DOUBLE;
-import static com.facebook.presto.hive.HiveType.HIVE_FLOAT;
-import static com.facebook.presto.hive.HiveType.HIVE_INT;
-import static com.facebook.presto.hive.HiveType.HIVE_LONG;
-import static com.facebook.presto.hive.HiveType.HIVE_SHORT;
-import static com.facebook.presto.hive.HiveType.HIVE_TIMESTAMP;
-import static com.facebook.presto.hive.HiveUtil.closeWithSuppression;
-import static com.facebook.presto.hive.HiveUtil.getTableObjectInspector;
-import static com.facebook.presto.hive.HiveUtil.isStructuralType;
-import static com.facebook.presto.hive.util.DecimalUtils.getLongDecimalValue;
-import static com.facebook.presto.hive.util.DecimalUtils.getShortDecimalValue;
-import static com.facebook.presto.hive.util.SerDeUtils.getBlockObject;
-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.Chars.isCharType;
-import static com.facebook.presto.spi.type.Chars.trimSpacesAndTruncateToLength;
-import static com.facebook.presto.spi.type.DateType.DATE;
-import static com.facebook.presto.spi.type.Decimals.isLongDecimal;
-import static com.facebook.presto.spi.type.Decimals.isShortDecimal;
-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.RealType.REAL;
-import static com.facebook.presto.spi.type.SmallintType.SMALLINT;
-import static com.facebook.presto.spi.type.StandardTypes.DECIMAL;
-import static com.facebook.presto.spi.type.TimestampType.TIMESTAMP;
-import static com.facebook.presto.spi.type.TinyintType.TINYINT;
-import static com.facebook.presto.spi.type.VarbinaryType.VARBINARY;
-import static com.facebook.presto.spi.type.Varchars.isVarcharType;
-import static com.facebook.presto.spi.type.Varchars.truncateToLength;
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkState;
-import static java.lang.Math.max;
-import static java.lang.Math.min;
-import static java.lang.String.format;
-import static java.util.Objects.requireNonNull;
-import static org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
-
-class ColumnarBinaryHiveRecordCursor
- implements RecordCursor
-{
- private final RecordReader recordReader;
- private final K key;
- private final BytesRefArrayWritable value;
-
- private final Type[] types;
- private final HiveType[] hiveTypes;
-
- private final ObjectInspector[] fieldInspectors; // DON'T USE THESE UNLESS EXTRACTION WILL BE SLOW ANYWAY
-
- private final int[] hiveColumnIndexes;
-
- private final boolean[] loaded;
- private final boolean[] booleans;
- private final long[] longs;
- private final double[] doubles;
- private final Slice[] slices;
- private final Object[] objects;
- private final boolean[] nulls;
-
- private final long totalBytes;
- private long completedBytes;
- private boolean closed;
-
- private final HiveDecimalWritable decimalWritable = new HiveDecimalWritable();
-
- private static final byte HIVE_EMPTY_STRING_BYTE = (byte) 0xbf;
-
- private static final int SIZE_OF_SHORT = 2;
- private static final int SIZE_OF_INT = 4;
- private static final int SIZE_OF_LONG = 8;
-
- private static final Set VALID_HIVE_STRING_TYPES = ImmutableSet.of(PrimitiveCategory.BINARY, PrimitiveCategory.VARCHAR, PrimitiveCategory.STRING);
- private static final Set VALID_HIVE_STRUCTURAL_CATEGORIES = ImmutableSet.of(Category.LIST, Category.MAP, Category.STRUCT);
-
- public ColumnarBinaryHiveRecordCursor(RecordReader recordReader,
- long totalBytes,
- Properties splitSchema,
- List columns,
- TypeManager typeManager)
- {
- requireNonNull(recordReader, "recordReader is null");
- checkArgument(totalBytes >= 0, "totalBytes is negative");
- requireNonNull(splitSchema, "splitSchema is null");
- requireNonNull(columns, "columns is null");
-
- this.recordReader = recordReader;
- this.totalBytes = totalBytes;
- this.key = recordReader.createKey();
- this.value = recordReader.createValue();
-
- int size = columns.size();
-
- this.types = new Type[size];
- this.hiveTypes = new HiveType[size];
-
- this.fieldInspectors = new ObjectInspector[size];
-
- this.hiveColumnIndexes = new int[size];
-
- this.loaded = new boolean[size];
- this.booleans = new boolean[size];
- this.longs = new long[size];
- this.doubles = new double[size];
- this.slices = new Slice[size];
- this.objects = new Object[size];
- this.nulls = new boolean[size];
-
- // initialize data columns
- StructObjectInspector rowInspector = getTableObjectInspector(splitSchema);
-
- for (int i = 0; i < columns.size(); i++) {
- HiveColumnHandle column = columns.get(i);
- checkState(column.getColumnType() == REGULAR, "column type must be regular");
-
- types[i] = typeManager.getType(column.getTypeSignature());
- hiveTypes[i] = column.getHiveType();
- hiveColumnIndexes[i] = column.getHiveColumnIndex();
-
- fieldInspectors[i] = rowInspector.getStructFieldRef(column.getName()).getFieldObjectInspector();
- }
- }
-
- @Override
- public long getTotalBytes()
- {
- return totalBytes;
- }
-
- @Override
- public long getCompletedBytes()
- {
- if (!closed) {
- updateCompletedBytes();
- }
- return completedBytes;
- }
-
- @Override
- public long getReadTimeNanos()
- {
- return 0;
- }
-
- private void updateCompletedBytes()
- {
- try {
- long newCompletedBytes = (long) (totalBytes * recordReader.getProgress());
- completedBytes = min(totalBytes, max(completedBytes, newCompletedBytes));
- }
- catch (IOException ignored) {
- }
- }
-
- @Override
- public Type getType(int field)
- {
- return types[field];
- }
-
- @Override
- public boolean advanceNextPosition()
- {
- try {
- if (closed || !recordReader.next(key, value)) {
- close();
- return false;
- }
-
- // reset loaded flags
- Arrays.fill(loaded, false);
-
- return true;
- }
- catch (IOException | RuntimeException e) {
- closeWithSuppression(this, e);
- throw new PrestoException(HIVE_CURSOR_ERROR, e);
- }
- }
-
- @Override
- public boolean getBoolean(int fieldId)
- {
- checkState(!closed, "Cursor is closed");
-
- validateType(fieldId, BOOLEAN);
- if (!loaded[fieldId]) {
- parseBooleanColumn(fieldId);
- }
- return booleans[fieldId];
- }
-
- private void parseBooleanColumn(int column)
- {
- loaded[column] = true;
-
- if (hiveColumnIndexes[column] >= value.size()) {
- // this partition may contain fewer fields than what's declared in the schema
- // this happens when additional columns are added to the hive table after a partition has been created
- nulls[column] = true;
- }
- else {
- BytesRefWritable fieldData = value.unCheckedGet(hiveColumnIndexes[column]);
-
- byte[] bytes;
- try {
- bytes = fieldData.getData();
- }
- catch (IOException e) {
- throw Throwables.propagate(e);
- }
-
- int start = fieldData.getStart();
- int length = fieldData.getLength();
-
- parseBooleanColumn(column, bytes, start, length);
- }
- }
-
- private void parseBooleanColumn(int column, byte[] bytes, int start, int length)
- {
- if (length > 0) {
- booleans[column] = bytes[start] != 0;
- nulls[column] = false;
- }
- else {
- nulls[column] = true;
- }
- }
-
- @Override
- public long getLong(int fieldId)
- {
- checkState(!closed, "Cursor is closed");
-
- if (!types[fieldId].equals(BIGINT) &&
- !types[fieldId].equals(INTEGER) &&
- !types[fieldId].equals(SMALLINT) &&
- !types[fieldId].equals(TINYINT) &&
- !types[fieldId].equals(DATE) &&
- !types[fieldId].equals(TIMESTAMP) &&
- !isShortDecimal(types[fieldId]) &&
- !types[fieldId].equals(REAL)) {
- // we don't use Preconditions.checkArgument because it requires boxing fieldId, which affects inner loop performance
- throw new IllegalArgumentException(
- format("Expected field to be %s, %s, %s, %s, %s, %s, %s or %s , actual %s (field %s)", TINYINT, SMALLINT, INTEGER, BIGINT, DATE, TIMESTAMP, DECIMAL, REAL, types[fieldId], fieldId));
- }
- if (!loaded[fieldId]) {
- parseLongColumn(fieldId);
- }
- return longs[fieldId];
- }
-
- private void parseLongColumn(int column)
- {
- loaded[column] = true;
-
- if (hiveColumnIndexes[column] >= value.size()) {
- // this partition may contain fewer fields than what's declared in the schema
- // this happens when additional columns are added to the hive table after a partition has been created
- nulls[column] = true;
- }
- else {
- BytesRefWritable fieldData = value.unCheckedGet(hiveColumnIndexes[column]);
-
- byte[] bytes;
- try {
- bytes = fieldData.getData();
- }
- catch (IOException e) {
- throw new PrestoException(HIVE_BAD_DATA, e);
- }
-
- int start = fieldData.getStart();
- int length = fieldData.getLength();
-
- parseLongColumn(column, bytes, start, length);
- }
- }
-
- private void parseLongColumn(int column, byte[] bytes, int start, int length)
- {
- if (length == 0) {
- nulls[column] = true;
- return;
- }
- nulls[column] = false;
- if (hiveTypes[column].equals(HIVE_SHORT)) {
- // the file format uses big endian
- checkState(length == SIZE_OF_SHORT, "Short should be 2 bytes");
- longs[column] = Short.reverseBytes(ByteArrays.getShort(bytes, start));
- }
- else if (hiveTypes[column].equals(HIVE_DATE)) {
- checkState(length >= 1, "Date should be at least 1 byte");
- long daysSinceEpoch = readVInt(bytes, start, length);
- longs[column] = daysSinceEpoch;
- }
- else if (hiveTypes[column].equals(HIVE_TIMESTAMP)) {
- checkState(length >= 1, "Timestamp should be at least 1 byte");
- long seconds = TimestampWritable.getSeconds(bytes, start);
- long nanos = (bytes[start] >> 7) != 0 ? TimestampWritable.getNanos(bytes, start + SIZE_OF_INT) : 0;
- longs[column] = (seconds * 1000) + (nanos / 1_000_000);
- }
- else if (hiveTypes[column].equals(HIVE_BYTE)) {
- checkState(length == 1, "Byte should be 1 byte");
- longs[column] = bytes[start];
- }
- else if (hiveTypes[column].equals(HIVE_INT)) {
- checkState(length >= 1, "Int should be at least 1 byte");
- if (length == 1) {
- longs[column] = bytes[start];
- }
- else {
- longs[column] = readVInt(bytes, start, length);
- }
- }
- else if (hiveTypes[column].equals(HIVE_LONG)) {
- checkState(length >= 1, "Long should be at least 1 byte");
- if (length == 1) {
- longs[column] = bytes[start];
- }
- else {
- longs[column] = readVInt(bytes, start, length);
- }
- }
- else if (hiveTypes[column].equals(HIVE_FLOAT)) {
- // the file format uses big endian
- checkState(length == SIZE_OF_INT, "Float should be 4 bytes");
- int intBits = ByteArrays.getInt(bytes, start);
- longs[column] = Integer.reverseBytes(intBits);
- }
- else {
- throw new RuntimeException(format("%s is not a valid LONG type", hiveTypes[column]));
- }
- }
-
- private static long readVInt(byte[] bytes, int start, int length)
- {
- long value = 0;
- for (int i = 1; i < length; i++) {
- value <<= 8;
- value |= (bytes[start + i] & 0xFF);
- }
- return WritableUtils.isNegativeVInt(bytes[start]) ? ~value : value;
- }
-
- @Override
- public double getDouble(int fieldId)
- {
- checkState(!closed, "Cursor is closed");
-
- validateType(fieldId, DOUBLE);
- if (!loaded[fieldId]) {
- parseDoubleColumn(fieldId);
- }
- return doubles[fieldId];
- }
-
- private void parseDoubleColumn(int column)
- {
- loaded[column] = true;
-
- if (hiveColumnIndexes[column] >= value.size()) {
- // this partition may contain fewer fields than what's declared in the schema
- // this happens when additional columns are added to the hive table after a partition has been created
- nulls[column] = true;
- }
- else {
- BytesRefWritable fieldData = value.unCheckedGet(hiveColumnIndexes[column]);
-
- byte[] bytes;
- try {
- bytes = fieldData.getData();
- }
- catch (IOException e) {
- throw Throwables.propagate(e);
- }
-
- int start = fieldData.getStart();
- int length = fieldData.getLength();
-
- parseDoubleColumn(column, bytes, start, length);
- }
- }
-
- private void parseDoubleColumn(int column, byte[] bytes, int start, int length)
- {
- if (length == 0) {
- nulls[column] = true;
- }
- else {
- checkState(hiveTypes[column].equals(HIVE_DOUBLE), "%s is not a valid DOUBLE type", hiveTypes[column]);
-
- nulls[column] = false;
- // the file format uses big endian
- checkState(length == SIZE_OF_LONG, "Double should be 8 bytes");
- long longBits = ByteArrays.getLong(bytes, start);
- doubles[column] = Double.longBitsToDouble(Long.reverseBytes(longBits));
- }
- }
-
- @Override
- public Slice getSlice(int fieldId)
- {
- checkState(!closed, "Cursor is closed");
-
- Type type = types[fieldId];
- if (!isVarcharType(type) && !isCharType(type) && !type.equals(VARBINARY) && !isStructuralType(hiveTypes[fieldId]) && !isLongDecimal(type)) {
- // we don't use Preconditions.checkArgument because it requires boxing fieldId, which affects inner loop performance
- throw new IllegalArgumentException(format("Expected field to be VARCHAR, CHAR, VARBINARY or DECIMAL, actual %s (field %s)", type, fieldId));
- }
-
- if (!loaded[fieldId]) {
- parseStringColumn(fieldId);
- }
- return slices[fieldId];
- }
-
- private void parseStringColumn(int column)
- {
- loaded[column] = true;
-
- if (hiveColumnIndexes[column] >= value.size()) {
- // this partition may contain fewer fields than what's declared in the schema
- // this happens when additional columns are added to the hive table after a partition has been created
- nulls[column] = true;
- }
- else {
- BytesRefWritable fieldData = value.unCheckedGet(hiveColumnIndexes[column]);
-
- byte[] bytes;
- try {
- bytes = fieldData.getData();
- }
- catch (IOException e) {
- throw Throwables.propagate(e);
- }
-
- int start = fieldData.getStart();
- int length = fieldData.getLength();
-
- parseStringColumn(column, bytes, start, length);
- }
- }
-
- private void parseStringColumn(int column, byte[] bytes, int start, int length)
- {
- checkState(isValidHiveStringType(hiveTypes[column]), "%s is not a valid STRING type", hiveTypes[column]);
- if (length == 0) {
- nulls[column] = true;
- }
- else {
- nulls[column] = false;
- // TODO: zero length BINARY is not supported. See https://issues.apache.org/jira/browse/HIVE-2483
- if (hiveTypes[column].equals(HiveType.HIVE_STRING) && (length == 1) && bytes[start] == HIVE_EMPTY_STRING_BYTE) {
- slices[column] = Slices.EMPTY_SLICE;
- }
- else {
- Slice value = Slices.wrappedBuffer(Arrays.copyOfRange(bytes, start, start + length));
- Type type = types[column];
- if (isVarcharType(type)) {
- slices[column] = truncateToLength(value, type);
- }
- else {
- slices[column] = value;
- }
- }
- }
- }
-
- private void parseCharColumn(int column)
- {
- loaded[column] = true;
-
- if (hiveColumnIndexes[column] >= value.size()) {
- // this partition may contain fewer fields than what's declared in the schema
- // this happens when additional columns are added to the hive table after a partition has been created
- nulls[column] = true;
- }
- else {
- BytesRefWritable fieldData = value.unCheckedGet(hiveColumnIndexes[column]);
-
- byte[] bytes;
- try {
- bytes = fieldData.getData();
- }
- catch (IOException e) {
- throw Throwables.propagate(e);
- }
-
- int start = fieldData.getStart();
- int length = fieldData.getLength();
-
- parseCharColumn(column, bytes, start, length);
- }
- }
-
- private void parseCharColumn(int column, byte[] bytes, int start, int length)
- {
- if (length == 0) {
- nulls[column] = true;
- }
- else {
- nulls[column] = false;
- Slice value = Slices.wrappedBuffer(Arrays.copyOfRange(bytes, start, start + length));
- Type type = types[column];
- slices[column] = trimSpacesAndTruncateToLength(value, type);
- }
- }
-
- private void parseDecimalColumn(int column)
- {
- loaded[column] = true;
-
- if (hiveColumnIndexes[column] >= value.size()) {
- // this partition may contain fewer fields than what's declared in the schema
- // this happens when additional columns are added to the hive table after a partition has been created
- nulls[column] = true;
- }
- else {
- BytesRefWritable fieldData = value.unCheckedGet(hiveColumnIndexes[column]);
-
- byte[] bytes;
- try {
- bytes = fieldData.getData();
- }
- catch (IOException e) {
- throw Throwables.propagate(e);
- }
-
- int start = fieldData.getStart();
- int length = fieldData.getLength();
-
- parseDecimalColumn(column, bytes, start, length);
- }
- }
-
- private void parseDecimalColumn(int column, byte[] bytes, int start, int length)
- {
- if (length == 0) {
- nulls[column] = true;
- }
- else {
- nulls[column] = false;
- decimalWritable.setFromBytes(bytes, start, length);
- DecimalType columnType = (DecimalType) types[column];
- if (columnType.isShort()) {
- longs[column] = getShortDecimalValue(decimalWritable, columnType.getScale());
- }
- else {
- slices[column] = getLongDecimalValue(decimalWritable, columnType.getScale());
- }
- }
- }
-
- private boolean isValidHiveStringType(HiveType hiveType)
- {
- return hiveType.getCategory() == Category.PRIMITIVE
- && VALID_HIVE_STRING_TYPES.contains(((PrimitiveTypeInfo) hiveType.getTypeInfo()).getPrimitiveCategory());
- }
-
- @Override
- public Object getObject(int fieldId)
- {
- checkState(!closed, "Cursor is closed");
-
- Type type = types[fieldId];
- if (!isStructuralType(hiveTypes[fieldId])) {
- // we don't use Preconditions.checkArgument because it requires boxing fieldId, which affects inner loop performance
- throw new IllegalArgumentException(format("Expected field to be structural, actual %s (field %s)", type, fieldId));
- }
-
- if (!loaded[fieldId]) {
- parseObjectColumn(fieldId);
- }
- return objects[fieldId];
- }
-
- private void parseObjectColumn(int column)
- {
- loaded[column] = true;
-
- if (hiveColumnIndexes[column] >= value.size()) {
- // this partition may contain fewer fields than what's declared in the schema
- // this happens when additional columns are added to the hive table after a partition has been created
- nulls[column] = true;
- }
- else {
- BytesRefWritable fieldData = value.unCheckedGet(hiveColumnIndexes[column]);
-
- byte[] bytes;
- try {
- bytes = fieldData.getData();
- }
- catch (IOException e) {
- throw Throwables.propagate(e);
- }
-
- int start = fieldData.getStart();
- int length = fieldData.getLength();
-
- parseObjectColumn(column, bytes, start, length);
- }
- }
-
- private void parseObjectColumn(int column, byte[] bytes, int start, int length)
- {
- checkState(VALID_HIVE_STRUCTURAL_CATEGORIES.contains(hiveTypes[column].getCategory()), "%s is not a valid STRUCTURAL type", hiveTypes[column]);
- if (length == 0) {
- nulls[column] = true;
- }
- else {
- nulls[column] = false;
- LazyBinaryObject extends ObjectInspector> lazyObject = LazyBinaryFactory.createLazyBinaryObject(fieldInspectors[column]);
- ByteArrayRef byteArrayRef = new ByteArrayRef();
- byteArrayRef.setData(bytes);
- lazyObject.init(byteArrayRef, start, length);
- objects[column] = getBlockObject(types[column], lazyObject.getObject(), fieldInspectors[column]);
- }
- }
-
- @Override
- public boolean isNull(int fieldId)
- {
- checkState(!closed, "Cursor is closed");
-
- if (!loaded[fieldId]) {
- parseColumn(fieldId);
- }
- return nulls[fieldId];
- }
-
- private void parseColumn(int column)
- {
- Type type = types[column];
- if (BOOLEAN.equals(type)) {
- parseBooleanColumn(column);
- }
- else if (BIGINT.equals(type)) {
- parseLongColumn(column);
- }
- else if (INTEGER.equals(type)) {
- parseLongColumn(column);
- }
- else if (SMALLINT.equals(type)) {
- parseLongColumn(column);
- }
- else if (TINYINT.equals(type)) {
- parseLongColumn(column);
- }
- else if (DOUBLE.equals(type)) {
- parseDoubleColumn(column);
- }
- else if (REAL.equals(type)) {
- parseLongColumn(column);
- }
- else if (isVarcharType(type) || VARBINARY.equals(type)) {
- parseStringColumn(column);
- }
- else if (isCharType(type)) {
- parseCharColumn(column);
- }
- else if (isStructuralType(hiveTypes[column])) {
- parseObjectColumn(column);
- }
- else if (DATE.equals(type)) {
- parseLongColumn(column);
- }
- else if (TIMESTAMP.equals(type)) {
- parseLongColumn(column);
- }
- else if (type instanceof DecimalType) {
- parseDecimalColumn(column);
- }
- else {
- throw new UnsupportedOperationException("Unsupported column type: " + type);
- }
- }
-
- private void validateType(int fieldId, Type type)
- {
- if (!types[fieldId].equals(type)) {
- // we don't use Preconditions.checkArgument because it requires boxing fieldId, which affects inner loop performance
- throw new IllegalArgumentException(format("Expected field to be %s, actual %s (field %s)", type, types[fieldId], fieldId));
- }
- }
-
- @Override
- public void close()
- {
- // some hive input formats are broken and bad things can happen if you close them multiple times
- if (closed) {
- return;
- }
- closed = true;
-
- updateCompletedBytes();
-
- try {
- recordReader.close();
- }
- catch (IOException e) {
- throw Throwables.propagate(e);
- }
- }
-}
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/ColumnarBinaryHiveRecordCursorProvider.java b/presto-hive/src/main/java/com/facebook/presto/hive/ColumnarBinaryHiveRecordCursorProvider.java
deleted file mode 100644
index 66879ad8424a4..0000000000000
--- a/presto-hive/src/main/java/com/facebook/presto/hive/ColumnarBinaryHiveRecordCursorProvider.java
+++ /dev/null
@@ -1,81 +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.hive;
-
-import com.facebook.presto.spi.ConnectorSession;
-import com.facebook.presto.spi.RecordCursor;
-import com.facebook.presto.spi.predicate.TupleDomain;
-import com.facebook.presto.spi.type.TypeManager;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable;
-import org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe;
-import org.apache.hadoop.mapred.RecordReader;
-import org.joda.time.DateTimeZone;
-
-import javax.inject.Inject;
-
-import java.util.List;
-import java.util.Optional;
-import java.util.Properties;
-
-import static com.facebook.presto.hive.HiveUtil.isDeserializerClass;
-import static java.util.Objects.requireNonNull;
-
-public class ColumnarBinaryHiveRecordCursorProvider
- implements HiveRecordCursorProvider
-{
- private final HdfsEnvironment hdfsEnvironment;
-
- @Inject
- public ColumnarBinaryHiveRecordCursorProvider(HdfsEnvironment hdfsEnvironment)
- {
- this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null");
- }
-
- @Override
- public Optional createRecordCursor(
- String clientId,
- Configuration configuration,
- ConnectorSession session,
- Path path,
- long start,
- long length,
- Properties schema,
- List columns,
- TupleDomain effectivePredicate,
- DateTimeZone hiveStorageTimeZone,
- TypeManager typeManager)
- {
- if (!isDeserializerClass(schema, LazyBinaryColumnarSerDe.class)) {
- return Optional.empty();
- }
-
- RecordReader, ?> recordReader = hdfsEnvironment.doAs(session.getUser(),
- () -> HiveUtil.createRecordReader(configuration, path, start, length, schema, columns));
-
- return Optional.of(new ColumnarBinaryHiveRecordCursor<>(
- bytesRecordReader(recordReader),
- length,
- schema,
- columns,
- typeManager));
- }
-
- @SuppressWarnings("unchecked")
- private static RecordReader, BytesRefArrayWritable> bytesRecordReader(RecordReader, ?> recordReader)
- {
- return (RecordReader, BytesRefArrayWritable>) recordReader;
- }
-}
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/ColumnarTextHiveRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/hive/ColumnarTextHiveRecordCursor.java
deleted file mode 100644
index 330dfbe936010..0000000000000
--- a/presto-hive/src/main/java/com/facebook/presto/hive/ColumnarTextHiveRecordCursor.java
+++ /dev/null
@@ -1,657 +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.hive;
-
-import com.facebook.presto.spi.PrestoException;
-import com.facebook.presto.spi.RecordCursor;
-import com.facebook.presto.spi.block.Block;
-import com.facebook.presto.spi.type.DecimalType;
-import com.facebook.presto.spi.type.Decimals;
-import com.facebook.presto.spi.type.Type;
-import com.facebook.presto.spi.type.TypeManager;
-import com.google.common.base.Throwables;
-import io.airlift.slice.Slice;
-import io.airlift.slice.Slices;
-import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable;
-import org.apache.hadoop.hive.serde2.columnar.BytesRefWritable;
-import org.apache.hadoop.hive.serde2.lazy.ByteArrayRef;
-import org.apache.hadoop.hive.serde2.lazy.LazyFactory;
-import org.apache.hadoop.hive.serde2.lazy.LazyObject;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
-import org.apache.hadoop.mapred.RecordReader;
-import org.joda.time.DateTimeZone;
-
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Properties;
-
-import static com.facebook.presto.hive.HiveBooleanParser.isFalse;
-import static com.facebook.presto.hive.HiveBooleanParser.isTrue;
-import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR;
-import static com.facebook.presto.hive.HiveDecimalParser.parseHiveDecimal;
-import static com.facebook.presto.hive.HiveErrorCode.HIVE_CURSOR_ERROR;
-import static com.facebook.presto.hive.HiveUtil.base64Decode;
-import static com.facebook.presto.hive.HiveUtil.closeWithSuppression;
-import static com.facebook.presto.hive.HiveUtil.getTableObjectInspector;
-import static com.facebook.presto.hive.HiveUtil.isStructuralType;
-import static com.facebook.presto.hive.HiveUtil.parseHiveDate;
-import static com.facebook.presto.hive.HiveUtil.parseHiveTimestamp;
-import static com.facebook.presto.hive.NumberParser.parseDouble;
-import static com.facebook.presto.hive.NumberParser.parseFloat;
-import static com.facebook.presto.hive.NumberParser.parseLong;
-import static com.facebook.presto.hive.util.SerDeUtils.getBlockObject;
-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.Chars.isCharType;
-import static com.facebook.presto.spi.type.Chars.trimSpacesAndTruncateToLength;
-import static com.facebook.presto.spi.type.DateType.DATE;
-import static com.facebook.presto.spi.type.Decimals.isShortDecimal;
-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.RealType.REAL;
-import static com.facebook.presto.spi.type.SmallintType.SMALLINT;
-import static com.facebook.presto.spi.type.StandardTypes.DECIMAL;
-import static com.facebook.presto.spi.type.TimestampType.TIMESTAMP;
-import static com.facebook.presto.spi.type.TinyintType.TINYINT;
-import static com.facebook.presto.spi.type.VarbinaryType.VARBINARY;
-import static com.facebook.presto.spi.type.Varchars.isVarcharType;
-import static com.facebook.presto.spi.type.Varchars.truncateToLength;
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkState;
-import static java.lang.Float.floatToRawIntBits;
-import static java.lang.Math.max;
-import static java.lang.Math.min;
-import static java.lang.String.format;
-import static java.util.Objects.requireNonNull;
-
-class ColumnarTextHiveRecordCursor
- implements RecordCursor
-{
- private final RecordReader recordReader;
- private final K key;
- private final BytesRefArrayWritable value;
-
- private final Type[] types;
- private final HiveType[] hiveTypes;
-
- private final ObjectInspector[] fieldInspectors; // DON'T USE THESE UNLESS EXTRACTION WILL BE SLOW ANYWAY
-
- private final int[] hiveColumnIndexes;
-
- private final boolean[] loaded;
- private final boolean[] booleans;
- private final long[] longs;
- private final double[] doubles;
- private final Slice[] slices;
- private final Object[] objects;
- private final boolean[] nulls;
-
- private final long totalBytes;
- private final DateTimeZone hiveStorageTimeZone;
-
- private long completedBytes;
- private boolean closed;
-
- public ColumnarTextHiveRecordCursor(
- RecordReader recordReader,
- long totalBytes,
- Properties splitSchema,
- List columns,
- DateTimeZone hiveStorageTimeZone,
- TypeManager typeManager)
- {
- requireNonNull(recordReader, "recordReader is null");
- checkArgument(totalBytes >= 0, "totalBytes is negative");
- requireNonNull(splitSchema, "splitSchema is null");
- requireNonNull(columns, "columns is null");
- requireNonNull(hiveStorageTimeZone, "hiveStorageTimeZone is null");
-
- this.recordReader = recordReader;
- this.totalBytes = totalBytes;
- this.key = recordReader.createKey();
- this.value = recordReader.createValue();
- this.hiveStorageTimeZone = hiveStorageTimeZone;
-
- int size = columns.size();
-
- this.types = new Type[size];
- this.hiveTypes = new HiveType[size];
-
- this.fieldInspectors = new ObjectInspector[size];
-
- this.hiveColumnIndexes = new int[size];
-
- this.loaded = new boolean[size];
- this.booleans = new boolean[size];
- this.longs = new long[size];
- this.doubles = new double[size];
- this.slices = new Slice[size];
- this.objects = new Object[size];
- this.nulls = new boolean[size];
-
- // initialize data columns
- StructObjectInspector rowInspector = getTableObjectInspector(splitSchema);
-
- for (int i = 0; i < columns.size(); i++) {
- HiveColumnHandle column = columns.get(i);
- checkState(column.getColumnType() == REGULAR, "column type must be regular");
-
- types[i] = typeManager.getType(column.getTypeSignature());
- hiveTypes[i] = column.getHiveType();
- hiveColumnIndexes[i] = column.getHiveColumnIndex();
-
- fieldInspectors[i] = rowInspector.getStructFieldRef(column.getName()).getFieldObjectInspector();
- }
- }
-
- @Override
- public long getTotalBytes()
- {
- return totalBytes;
- }
-
- @Override
- public long getCompletedBytes()
- {
- if (!closed) {
- updateCompletedBytes();
- }
- return completedBytes;
- }
-
- @Override
- public long getReadTimeNanos()
- {
- return 0;
- }
-
- private void updateCompletedBytes()
- {
- try {
- long newCompletedBytes = (long) (totalBytes * recordReader.getProgress());
- completedBytes = min(totalBytes, max(completedBytes, newCompletedBytes));
- }
- catch (IOException ignored) {
- }
- }
-
- @Override
- public Type getType(int field)
- {
- return types[field];
- }
-
- @Override
- public boolean advanceNextPosition()
- {
- try {
- if (closed || !recordReader.next(key, value)) {
- close();
- return false;
- }
-
- // reset loaded flags
- Arrays.fill(loaded, false);
-
- return true;
- }
- catch (IOException | RuntimeException e) {
- closeWithSuppression(this, e);
- throw new PrestoException(HIVE_CURSOR_ERROR, e);
- }
- }
-
- @Override
- public boolean getBoolean(int fieldId)
- {
- checkState(!closed, "Cursor is closed");
-
- validateType(fieldId, boolean.class);
- if (!loaded[fieldId]) {
- parseBooleanColumn(fieldId);
- }
- return booleans[fieldId];
- }
-
- private void parseBooleanColumn(int column)
- {
- loaded[column] = true;
-
- if (hiveColumnIndexes[column] >= value.size()) {
- // this partition may contain fewer fields than what's declared in the schema
- // this happens when additional columns are added to the hive table after a partition has been created
- nulls[column] = true;
- }
- else {
- BytesRefWritable fieldData = value.unCheckedGet(hiveColumnIndexes[column]);
-
- byte[] bytes;
- try {
- bytes = fieldData.getData();
- }
- catch (IOException e) {
- throw Throwables.propagate(e);
- }
-
- int start = fieldData.getStart();
- int length = fieldData.getLength();
-
- parseBooleanColumn(column, bytes, start, length);
- }
- }
-
- private void parseBooleanColumn(int column, byte[] bytes, int start, int length)
- {
- boolean wasNull;
- if (isTrue(bytes, start, length)) {
- booleans[column] = true;
- wasNull = false;
- }
- else if (isFalse(bytes, start, length)) {
- booleans[column] = false;
- wasNull = false;
- }
- else {
- wasNull = true;
- }
- nulls[column] = wasNull;
- }
-
- @Override
- public long getLong(int fieldId)
- {
- checkState(!closed, "Cursor is closed");
-
- if (!types[fieldId].equals(BIGINT) &&
- !types[fieldId].equals(INTEGER) &&
- !types[fieldId].equals(SMALLINT) &&
- !types[fieldId].equals(TINYINT) &&
- !types[fieldId].equals(DATE) &&
- !types[fieldId].equals(TIMESTAMP) &&
- !isShortDecimal(types[fieldId]) &&
- !types[fieldId].equals(REAL)) {
- // we don't use Preconditions.checkArgument because it requires boxing fieldId, which affects inner loop performance
- throw new IllegalArgumentException(
- format("Expected field to be %s, %s, %s, %s, %s, %s, %s or %s , actual %s (field %s)", TINYINT, SMALLINT, INTEGER, BIGINT, DECIMAL, DATE, TIMESTAMP, REAL, types[fieldId], fieldId));
- }
-
- if (!loaded[fieldId]) {
- parseLongColumn(fieldId);
- }
- return longs[fieldId];
- }
-
- private void parseLongColumn(int column)
- {
- loaded[column] = true;
-
- if (hiveColumnIndexes[column] >= value.size()) {
- // this partition may contain fewer fields than what's declared in the schema
- // this happens when additional columns are added to the hive table after a partition has been created
- nulls[column] = true;
- }
- else {
- BytesRefWritable fieldData = value.unCheckedGet(hiveColumnIndexes[column]);
-
- byte[] bytes;
- try {
- bytes = fieldData.getData();
- }
- catch (IOException e) {
- throw Throwables.propagate(e);
- }
-
- int start = fieldData.getStart();
- int length = fieldData.getLength();
-
- parseLongColumn(column, bytes, start, length);
- }
- }
-
- private void parseLongColumn(int column, byte[] bytes, int start, int length)
- {
- boolean wasNull;
- if (length == 0 || (length == "\\N".length() && bytes[start] == '\\' && bytes[start + 1] == 'N')) {
- wasNull = true;
- }
- else if (hiveTypes[column].equals(HiveType.HIVE_DATE)) {
- String value = new String(bytes, start, length);
- longs[column] = parseHiveDate(value);
- wasNull = false;
- }
- else if (hiveTypes[column].equals(HiveType.HIVE_TIMESTAMP)) {
- String value = new String(bytes, start, length);
- longs[column] = parseHiveTimestamp(value, hiveStorageTimeZone);
- wasNull = false;
- }
- else if (hiveTypes[column].equals(HiveType.HIVE_FLOAT)) {
- longs[column] = floatToRawIntBits(parseFloat(bytes, start, length));
- wasNull = false;
- }
- else {
- longs[column] = parseLong(bytes, start, length);
- wasNull = false;
- }
- nulls[column] = wasNull;
- }
-
- @Override
- public double getDouble(int fieldId)
- {
- checkState(!closed, "Cursor is closed");
-
- validateType(fieldId, double.class);
- if (!loaded[fieldId]) {
- parseDoubleColumn(fieldId);
- }
- return doubles[fieldId];
- }
-
- private void parseDoubleColumn(int column)
- {
- loaded[column] = true;
-
- if (hiveColumnIndexes[column] >= value.size()) {
- // this partition may contain fewer fields than what's declared in the schema
- // this happens when additional columns are added to the hive table after a partition has been created
- nulls[column] = true;
- }
- else {
- BytesRefWritable fieldData = value.unCheckedGet(hiveColumnIndexes[column]);
-
- byte[] bytes;
- try {
- bytes = fieldData.getData();
- }
- catch (IOException e) {
- throw Throwables.propagate(e);
- }
-
- int start = fieldData.getStart();
- int length = fieldData.getLength();
-
- parseDoubleColumn(column, bytes, start, length);
- }
- }
-
- private void parseDoubleColumn(int column, byte[] bytes, int start, int length)
- {
- boolean wasNull;
- if (length == 0 || (length == "\\N".length() && bytes[start] == '\\' && bytes[start + 1] == 'N')) {
- wasNull = true;
- }
- else {
- doubles[column] = parseDouble(bytes, start, length);
- wasNull = false;
- }
- nulls[column] = wasNull;
- }
-
- @Override
- public Slice getSlice(int fieldId)
- {
- checkState(!closed, "Cursor is closed");
-
- validateType(fieldId, Slice.class);
- if (!loaded[fieldId]) {
- parseStringColumn(fieldId);
- }
- return slices[fieldId];
- }
-
- private void parseStringColumn(int column)
- {
- loaded[column] = true;
-
- if (hiveColumnIndexes[column] >= value.size()) {
- // this partition may contain fewer fields than what's declared in the schema
- // this happens when additional columns are added to the hive table after a partition has been created
- nulls[column] = true;
- }
- else {
- BytesRefWritable fieldData = value.unCheckedGet(hiveColumnIndexes[column]);
-
- byte[] bytes;
- try {
- bytes = fieldData.getData();
- }
- catch (IOException e) {
- throw Throwables.propagate(e);
- }
-
- int start = fieldData.getStart();
- int length = fieldData.getLength();
-
- parseStringColumn(column, bytes, start, length);
- }
- }
-
- private void parseStringColumn(int column, byte[] bytes, int start, int length)
- {
- boolean wasNull;
- if (length == "\\N".length() && bytes[start] == '\\' && bytes[start + 1] == 'N') {
- wasNull = true;
- }
- else {
- Type type = types[column];
- Slice value = Slices.wrappedBuffer(Arrays.copyOfRange(bytes, start, start + length));
- if (isVarcharType(type)) {
- slices[column] = truncateToLength(value, type);
- }
- else if (isCharType(type)) {
- slices[column] = trimSpacesAndTruncateToLength(value, type);
- }
- // this is unbelievably stupid but Hive base64 encodes binary data in a binary file format
- else if (type.equals(VARBINARY)) {
- // and yes we end up with an extra copy here because the Base64 only handles whole arrays
- slices[column] = base64Decode(value.getBytes());
- }
- else {
- slices[column] = value;
- }
- wasNull = false;
- }
- nulls[column] = wasNull;
- }
-
- private void parseDecimalColumn(int column)
- {
- loaded[column] = true;
-
- if (hiveColumnIndexes[column] >= value.size()) {
- // this partition may contain fewer fields than what's declared in the schema
- // this happens when additional columns are added to the hive table after a partition has been created
- nulls[column] = true;
- }
- else {
- BytesRefWritable fieldData = value.unCheckedGet(hiveColumnIndexes[column]);
-
- byte[] bytes;
- try {
- bytes = fieldData.getData();
- }
- catch (IOException e) {
- throw Throwables.propagate(e);
- }
-
- int start = fieldData.getStart();
- int length = fieldData.getLength();
-
- parseDecimalColumn(column, bytes, start, length);
- }
- }
-
- private void parseDecimalColumn(int column, byte[] bytes, int start, int length)
- {
- boolean wasNull;
- if (length == 0 || (length == "\\N".length() && bytes[start] == '\\' && bytes[start + 1] == 'N')) {
- wasNull = true;
- }
- else {
- DecimalType columnType = (DecimalType) types[column];
- BigDecimal decimal = parseHiveDecimal(bytes, start, length, columnType);
-
- if (columnType.isShort()) {
- longs[column] = decimal.unscaledValue().longValue();
- }
- else {
- slices[column] = Decimals.encodeUnscaledValue(decimal.unscaledValue());
- }
-
- wasNull = false;
- }
- nulls[column] = wasNull;
- }
-
- @Override
- public Object getObject(int fieldId)
- {
- checkState(!closed, "Cursor is closed");
-
- validateType(fieldId, Block.class);
- if (!loaded[fieldId]) {
- parseObjectColumn(fieldId);
- }
- return objects[fieldId];
- }
-
- private void parseObjectColumn(int column)
- {
- loaded[column] = true;
-
- if (hiveColumnIndexes[column] >= value.size()) {
- // this partition may contain fewer fields than what's declared in the schema
- // this happens when additional columns are added to the hive table after a partition has been created
- nulls[column] = true;
- }
- else {
- BytesRefWritable fieldData = value.unCheckedGet(hiveColumnIndexes[column]);
-
- byte[] bytes;
- try {
- bytes = fieldData.getData();
- }
- catch (IOException e) {
- throw Throwables.propagate(e);
- }
-
- int start = fieldData.getStart();
- int length = fieldData.getLength();
-
- parseObjectColumn(column, bytes, start, length);
- }
- }
-
- private void parseObjectColumn(int column, byte[] bytes, int start, int length)
- {
- boolean wasNull;
- if (length == "\\N".length() && bytes[start] == '\\' && bytes[start + 1] == 'N') {
- wasNull = true;
- }
- else {
- LazyObject extends ObjectInspector> lazyObject = LazyFactory.createLazyObject(fieldInspectors[column]);
- ByteArrayRef byteArrayRef = new ByteArrayRef();
- byteArrayRef.setData(bytes);
- lazyObject.init(byteArrayRef, start, length);
- objects[column] = getBlockObject(types[column], lazyObject.getObject(), fieldInspectors[column]);
- wasNull = false;
- }
- nulls[column] = wasNull;
- }
-
- @Override
- public boolean isNull(int fieldId)
- {
- checkState(!closed, "Cursor is closed");
-
- if (!loaded[fieldId]) {
- parseColumn(fieldId);
- }
- return nulls[fieldId];
- }
-
- private void parseColumn(int column)
- {
- Type type = types[column];
- if (type.equals(BOOLEAN)) {
- parseBooleanColumn(column);
- }
- else if (type.equals(BIGINT)) {
- parseLongColumn(column);
- }
- else if (type.equals(INTEGER)) {
- parseLongColumn(column);
- }
- else if (type.equals(SMALLINT)) {
- parseLongColumn(column);
- }
- else if (type.equals(TINYINT)) {
- parseLongColumn(column);
- }
- else if (type.equals(REAL)) {
- parseLongColumn(column);
- }
- else if (type.equals(DOUBLE)) {
- parseDoubleColumn(column);
- }
- else if (isVarcharType(type) || VARBINARY.equals(type) || isCharType(type)) {
- parseStringColumn(column);
- }
- else if (isStructuralType(hiveTypes[column])) {
- parseObjectColumn(column);
- }
- else if (type.equals(DATE)) {
- parseLongColumn(column);
- }
- else if (type.equals(TIMESTAMP)) {
- parseLongColumn(column);
- }
- else if (type instanceof DecimalType) {
- parseDecimalColumn(column);
- }
- else {
- throw new UnsupportedOperationException("Unsupported column type: " + type);
- }
- }
-
- private void validateType(int fieldId, Class> type)
- {
- if (!types[fieldId].getJavaType().equals(type)) {
- // we don't use Preconditions.checkArgument because it requires boxing fieldId, which affects inner loop performance
- throw new IllegalArgumentException(String.format("Expected field to be %s, actual %s (field %s)", type, types[fieldId], fieldId));
- }
- }
-
- @Override
- public void close()
- {
- // some hive input formats are broken and bad things can happen if you close them multiple times
- if (closed) {
- return;
- }
- closed = true;
-
- updateCompletedBytes();
-
- try {
- recordReader.close();
- }
- catch (IOException e) {
- throw Throwables.propagate(e);
- }
- }
-}
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/ColumnarTextHiveRecordCursorProvider.java b/presto-hive/src/main/java/com/facebook/presto/hive/ColumnarTextHiveRecordCursorProvider.java
deleted file mode 100644
index 7618b459cced7..0000000000000
--- a/presto-hive/src/main/java/com/facebook/presto/hive/ColumnarTextHiveRecordCursorProvider.java
+++ /dev/null
@@ -1,82 +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.hive;
-
-import com.facebook.presto.spi.ConnectorSession;
-import com.facebook.presto.spi.RecordCursor;
-import com.facebook.presto.spi.predicate.TupleDomain;
-import com.facebook.presto.spi.type.TypeManager;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable;
-import org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe;
-import org.apache.hadoop.mapred.RecordReader;
-import org.joda.time.DateTimeZone;
-
-import javax.inject.Inject;
-
-import java.util.List;
-import java.util.Optional;
-import java.util.Properties;
-
-import static com.facebook.presto.hive.HiveUtil.isDeserializerClass;
-import static java.util.Objects.requireNonNull;
-
-public class ColumnarTextHiveRecordCursorProvider
- implements HiveRecordCursorProvider
-{
- private final HdfsEnvironment hdfsEnvironment;
-
- @Inject
- public ColumnarTextHiveRecordCursorProvider(HdfsEnvironment hdfsEnvironment)
- {
- this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null");
- }
-
- @Override
- public Optional createRecordCursor(
- String clientId,
- Configuration configuration,
- ConnectorSession session,
- Path path,
- long start,
- long length,
- Properties schema,
- List columns,
- TupleDomain effectivePredicate,
- DateTimeZone hiveStorageTimeZone,
- TypeManager typeManager)
- {
- if (!isDeserializerClass(schema, ColumnarSerDe.class)) {
- return Optional.empty();
- }
-
- RecordReader, ?> recordReader = hdfsEnvironment.doAs(session.getUser(),
- () -> HiveUtil.createRecordReader(configuration, path, start, length, schema, columns));
-
- return Optional.of(new ColumnarTextHiveRecordCursor<>(
- columnarTextRecordReader(recordReader),
- length,
- schema,
- columns,
- hiveStorageTimeZone,
- typeManager));
- }
-
- @SuppressWarnings("unchecked")
- private static RecordReader, BytesRefArrayWritable> columnarTextRecordReader(RecordReader, ?> recordReader)
- {
- return (RecordReader, BytesRefArrayWritable>) recordReader;
- }
-}
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/FileFormatDataSourceStats.java b/presto-hive/src/main/java/com/facebook/presto/hive/FileFormatDataSourceStats.java
new file mode 100644
index 0000000000000..1184b19464ffd
--- /dev/null
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/FileFormatDataSourceStats.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.hive;
+
+import io.airlift.stats.DistributionStat;
+import io.airlift.stats.TimeStat;
+import org.weakref.jmx.Managed;
+import org.weakref.jmx.Nested;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+public class FileFormatDataSourceStats
+{
+ private final DistributionStat readBytes = new DistributionStat();
+ private final DistributionStat loadedBlockBytes = new DistributionStat();
+ private final DistributionStat maxCombinedBytesPerRow = new DistributionStat();
+ private final TimeStat time0Bto100KB = new TimeStat(MILLISECONDS);
+ private final TimeStat time100KBto1MB = new TimeStat(MILLISECONDS);
+ private final TimeStat time1MBto10MB = new TimeStat(MILLISECONDS);
+ private final TimeStat time10MBPlus = new TimeStat(MILLISECONDS);
+
+ @Managed
+ @Nested
+ public DistributionStat getReadBytes()
+ {
+ return readBytes;
+ }
+
+ @Managed
+ @Nested
+ public DistributionStat getLoadedBlockBytes()
+ {
+ return loadedBlockBytes;
+ }
+
+ @Managed
+ @Nested
+ public DistributionStat getMaxCombinedBytesPerRow()
+ {
+ return maxCombinedBytesPerRow;
+ }
+
+ @Managed
+ @Nested
+ public TimeStat get0Bto100KB()
+ {
+ return time0Bto100KB;
+ }
+
+ @Managed
+ @Nested
+ public TimeStat get100KBto1MB()
+ {
+ return time100KBto1MB;
+ }
+
+ @Managed
+ @Nested
+ public TimeStat get1MBto10MB()
+ {
+ return time1MBto10MB;
+ }
+
+ @Managed
+ @Nested
+ public TimeStat get10MBPlus()
+ {
+ return time10MBPlus;
+ }
+
+ public void readDataBytesPerSecond(long bytes, long nanos)
+ {
+ readBytes.add(bytes);
+ if (bytes < 100 * 1024) {
+ time0Bto100KB.add(nanos, NANOSECONDS);
+ }
+ else if (bytes < 1024 * 1024) {
+ time100KBto1MB.add(nanos, NANOSECONDS);
+ }
+ else if (bytes < 10 * 1024 * 1024) {
+ time1MBto10MB.add(nanos, NANOSECONDS);
+ }
+ else {
+ time10MBPlus.add(nanos, NANOSECONDS);
+ }
+ }
+
+ public void addLoadedBlockSize(long bytes)
+ {
+ loadedBlockBytes.add(bytes);
+ }
+
+ public void addMaxCombinedBytesPerRow(long bytes)
+ {
+ maxCombinedBytesPerRow.add(bytes);
+ }
+}
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/GenericHiveRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/hive/GenericHiveRecordCursor.java
index 87ce874498406..0b2c739f8e38c 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/GenericHiveRecordCursor.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/GenericHiveRecordCursor.java
@@ -23,15 +23,18 @@
import com.google.common.base.Throwables;
import io.airlift.slice.Slice;
import io.airlift.slice.Slices;
-import org.apache.hadoop.hive.common.type.HiveChar;
import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hadoop.hive.common.type.HiveVarchar;
import org.apache.hadoop.hive.serde2.Deserializer;
import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
+import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.StructField;
import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.io.BinaryComparable;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapred.RecordReader;
import org.joda.time.DateTimeZone;
@@ -359,24 +362,27 @@ private void parseStringColumn(int column)
nulls[column] = true;
}
else {
- Object fieldValue = ((PrimitiveObjectInspector) fieldInspectors[column]).getPrimitiveJavaObject(fieldData);
+ Object fieldValue = ((PrimitiveObjectInspector) fieldInspectors[column]).getPrimitiveWritableObject(fieldData);
checkState(fieldValue != null, "fieldValue should not be null");
- Slice value;
- if (fieldValue instanceof String) {
- value = Slices.utf8Slice((String) fieldValue);
+ BinaryComparable hiveValue;
+ if (fieldValue instanceof Text) {
+ hiveValue = (Text) fieldValue;
}
- else if (fieldValue instanceof byte[]) {
- value = Slices.wrappedBuffer((byte[]) fieldValue);
+ else if (fieldValue instanceof BytesWritable) {
+ hiveValue = (BytesWritable) fieldValue;
}
- else if (fieldValue instanceof HiveVarchar) {
- value = Slices.utf8Slice(((HiveVarchar) fieldValue).getValue());
+ else if (fieldValue instanceof HiveVarcharWritable) {
+ hiveValue = ((HiveVarcharWritable) fieldValue).getTextValue();
}
- else if (fieldValue instanceof HiveChar) {
- value = Slices.utf8Slice(((HiveChar) fieldValue).getValue());
+ else if (fieldValue instanceof HiveCharWritable) {
+ hiveValue = ((HiveCharWritable) fieldValue).getTextValue();
}
else {
throw new IllegalStateException("unsupported string field type: " + fieldValue.getClass().getName());
}
+
+ // create a slice view over the hive value and trim to character limits
+ Slice value = Slices.wrappedBuffer(hiveValue.getBytes(), 0, hiveValue.getLength());
Type type = types[column];
if (isVarcharType(type)) {
value = truncateToLength(value, type);
@@ -384,7 +390,9 @@ else if (fieldValue instanceof HiveChar) {
if (isCharType(type)) {
value = trimSpacesAndTruncateToLength(value, type);
}
- slices[column] = value;
+
+ // store a copy of the bytes, since the hive reader can reuse the underlying buffer
+ slices[column] = Slices.copyOf(value);
nulls[column] = false;
}
}
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HdfsConfigurationUpdater.java b/presto-hive/src/main/java/com/facebook/presto/hive/HdfsConfigurationUpdater.java
index fb38275f51fc8..6b277f4141296 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/HdfsConfigurationUpdater.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/HdfsConfigurationUpdater.java
@@ -133,7 +133,7 @@ private static Configuration readConfiguration(List resourcePaths)
return result;
}
- public void updateConfiguration(PrestoHadoopConfiguration config)
+ public void updateConfiguration(Configuration config)
{
copy(resourcesConfiguration, config);
@@ -251,5 +251,11 @@ public void reloadCachedMappings()
{
// no-op
}
+
+ @Override
+ public void reloadCachedMappings(List names)
+ {
+ // no-op
+ }
}
}
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveClientConfig.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveClientConfig.java
index bea8e12b58e85..716d9414bd647 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveClientConfig.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveClientConfig.java
@@ -96,19 +96,14 @@ public class HiveClientConfig
private DataSize orcMaxMergeDistance = new DataSize(1, MEGABYTE);
private DataSize orcMaxBufferSize = new DataSize(8, MEGABYTE);
private DataSize orcStreamBufferSize = new DataSize(8, MEGABYTE);
+ private DataSize orcMaxReadBlockSize = new DataSize(16, MEGABYTE);
- private boolean rcfileOptimizedReaderEnabled = true;
- private boolean rcfileOptimizedWriterEnabled;
+ private boolean rcfileOptimizedWriterEnabled = true;
+ private boolean rcfileWriterValidate;
private HiveMetastoreAuthenticationType hiveMetastoreAuthenticationType = HiveMetastoreAuthenticationType.NONE;
- private String hiveMetastoreServicePrincipal;
- private String hiveMetastoreClientPrincipal;
- private String hiveMetastoreClientKeytab;
-
private HdfsAuthenticationType hdfsAuthenticationType = HdfsAuthenticationType.NONE;
private boolean hdfsImpersonationEnabled;
- private String hdfsPrestoPrincipal;
- private String hdfsPrestoKeytab;
private boolean skipDeletionForAlter;
@@ -656,29 +651,28 @@ public HiveClientConfig setOrcStreamBufferSize(DataSize orcStreamBufferSize)
return this;
}
- public boolean isOrcBloomFiltersEnabled()
+ @NotNull
+ public DataSize getOrcMaxReadBlockSize()
{
- return orcBloomFiltersEnabled;
+ return orcMaxReadBlockSize;
}
- @Config("hive.orc.bloom-filters.enabled")
- public HiveClientConfig setOrcBloomFiltersEnabled(boolean orcBloomFiltersEnabled)
+ @Config("hive.orc.max-read-block-size")
+ public HiveClientConfig setOrcMaxReadBlockSize(DataSize orcMaxReadBlockSize)
{
- this.orcBloomFiltersEnabled = orcBloomFiltersEnabled;
+ this.orcMaxReadBlockSize = orcMaxReadBlockSize;
return this;
}
- @Deprecated
- public boolean isRcfileOptimizedReaderEnabled()
+ public boolean isOrcBloomFiltersEnabled()
{
- return rcfileOptimizedReaderEnabled;
+ return orcBloomFiltersEnabled;
}
- @Deprecated
- @Config("hive.rcfile-optimized-reader.enabled")
- public HiveClientConfig setRcfileOptimizedReaderEnabled(boolean rcfileOptimizedReaderEnabled)
+ @Config("hive.orc.bloom-filters.enabled")
+ public HiveClientConfig setOrcBloomFiltersEnabled(boolean orcBloomFiltersEnabled)
{
- this.rcfileOptimizedReaderEnabled = rcfileOptimizedReaderEnabled;
+ this.orcBloomFiltersEnabled = orcBloomFiltersEnabled;
return this;
}
@@ -696,6 +690,19 @@ public HiveClientConfig setRcfileOptimizedWriterEnabled(boolean rcfileOptimizedW
return this;
}
+ public boolean isRcfileWriterValidate()
+ {
+ return rcfileWriterValidate;
+ }
+
+ @Config("hive.rcfile.writer.validate")
+ @ConfigDescription("Validate RCFile after write by re-reading the whole file")
+ public HiveClientConfig setRcfileWriterValidate(boolean rcfileWriterValidate)
+ {
+ this.rcfileWriterValidate = rcfileWriterValidate;
+ return this;
+ }
+
public boolean isAssumeCanonicalPartitionKeys()
{
return assumeCanonicalPartitionKeys;
@@ -727,6 +734,7 @@ public enum HiveMetastoreAuthenticationType
KERBEROS
}
+ @NotNull
public HiveMetastoreAuthenticationType getHiveMetastoreAuthenticationType()
{
return hiveMetastoreAuthenticationType;
@@ -740,51 +748,13 @@ public HiveClientConfig setHiveMetastoreAuthenticationType(HiveMetastoreAuthenti
return this;
}
- public String getHiveMetastoreServicePrincipal()
- {
- return hiveMetastoreServicePrincipal;
- }
-
- @Config("hive.metastore.service.principal")
- @ConfigDescription("Hive Metastore service principal")
- public HiveClientConfig setHiveMetastoreServicePrincipal(String hiveMetastoreServicePrincipal)
- {
- this.hiveMetastoreServicePrincipal = hiveMetastoreServicePrincipal;
- return this;
- }
-
- public String getHiveMetastoreClientPrincipal()
- {
- return hiveMetastoreClientPrincipal;
- }
-
- @Config("hive.metastore.client.principal")
- @ConfigDescription("Hive Metastore client principal")
- public HiveClientConfig setHiveMetastoreClientPrincipal(String hiveMetastoreClientPrincipal)
- {
- this.hiveMetastoreClientPrincipal = hiveMetastoreClientPrincipal;
- return this;
- }
-
- public String getHiveMetastoreClientKeytab()
- {
- return hiveMetastoreClientKeytab;
- }
-
- @Config("hive.metastore.client.keytab")
- @ConfigDescription("Hive Metastore client keytab location")
- public HiveClientConfig setHiveMetastoreClientKeytab(String hiveMetastoreClientKeytab)
- {
- this.hiveMetastoreClientKeytab = hiveMetastoreClientKeytab;
- return this;
- }
-
public enum HdfsAuthenticationType
{
NONE,
KERBEROS,
}
+ @NotNull
public HdfsAuthenticationType getHdfsAuthenticationType()
{
return hdfsAuthenticationType;
@@ -811,32 +781,6 @@ public HiveClientConfig setHdfsImpersonationEnabled(boolean hdfsImpersonationEna
return this;
}
- public String getHdfsPrestoPrincipal()
- {
- return hdfsPrestoPrincipal;
- }
-
- @Config("hive.hdfs.presto.principal")
- @ConfigDescription("Presto principal used to access HDFS")
- public HiveClientConfig setHdfsPrestoPrincipal(String hdfsPrestoPrincipal)
- {
- this.hdfsPrestoPrincipal = hdfsPrestoPrincipal;
- return this;
- }
-
- public String getHdfsPrestoKeytab()
- {
- return hdfsPrestoKeytab;
- }
-
- @Config("hive.hdfs.presto.keytab")
- @ConfigDescription("Presto keytab used to access HDFS")
- public HiveClientConfig setHdfsPrestoKeytab(String hdfsPrestoKeytab)
- {
- this.hdfsPrestoKeytab = hdfsPrestoKeytab;
- return this;
- }
-
public boolean isSkipDeletionForAlter()
{
return skipDeletionForAlter;
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveClientModule.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveClientModule.java
index 5af9598f9dc9b..d9bd8a01bbbe4 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveClientModule.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveClientModule.java
@@ -81,7 +81,7 @@ public void configure(Binder binder)
binder.bind(HiveTableProperties.class).in(Scopes.SINGLETON);
binder.bind(NamenodeStats.class).in(Scopes.SINGLETON);
- newExporter(binder).export(NamenodeStats.class).as(generatedNameOf(NamenodeStats.class));
+ newExporter(binder).export(NamenodeStats.class).as(generatedNameOf(NamenodeStats.class, connectorId));
binder.bind(HiveMetastoreClientFactory.class).in(Scopes.SINGLETON);
binder.bind(HiveCluster.class).to(StaticHiveCluster.class).in(Scopes.SINGLETON);
@@ -93,8 +93,6 @@ public void configure(Binder binder)
Multibinder recordCursorProviderBinder = newSetBinder(binder, HiveRecordCursorProvider.class);
recordCursorProviderBinder.addBinding().to(ParquetRecordCursorProvider.class).in(Scopes.SINGLETON);
- recordCursorProviderBinder.addBinding().to(ColumnarTextHiveRecordCursorProvider.class).in(Scopes.SINGLETON);
- recordCursorProviderBinder.addBinding().to(ColumnarBinaryHiveRecordCursorProvider.class).in(Scopes.SINGLETON);
recordCursorProviderBinder.addBinding().to(GenericHiveRecordCursorProvider.class).in(Scopes.SINGLETON);
newSetBinder(binder, EventClient.class).addBinding().to(HiveEventClient.class).in(Scopes.SINGLETON);
@@ -110,6 +108,9 @@ public void configure(Binder binder)
jsonCodecBinder(binder).bindJsonCodec(PartitionUpdate.class);
+ binder.bind(FileFormatDataSourceStats.class).in(Scopes.SINGLETON);
+ newExporter(binder).export(FileFormatDataSourceStats.class).as(generatedNameOf(FileFormatDataSourceStats.class, connectorId));
+
Multibinder pageSourceFactoryBinder = newSetBinder(binder, HivePageSourceFactory.class);
pageSourceFactoryBinder.addBinding().to(OrcPageSourceFactory.class).in(Scopes.SINGLETON);
pageSourceFactoryBinder.addBinding().to(DwrfPageSourceFactory.class).in(Scopes.SINGLETON);
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveHdfsConfiguration.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveHdfsConfiguration.java
index e48b429c6c859..23563bf47d0c0 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveHdfsConfiguration.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveHdfsConfiguration.java
@@ -13,7 +13,6 @@
*/
package com.facebook.presto.hive;
-import com.google.common.collect.ImmutableClassToInstanceMap;
import org.apache.hadoop.conf.Configuration;
import javax.inject.Inject;
@@ -45,7 +44,7 @@ public class HiveHdfsConfiguration
@Override
protected Configuration initialValue()
{
- PrestoHadoopConfiguration configuration = new PrestoHadoopConfiguration(ImmutableClassToInstanceMap.of());
+ Configuration configuration = new Configuration(false);
copy(INITIAL_CONFIGURATION, configuration);
updater.updateConfiguration(configuration);
return configuration;
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java
index 5703ec7c1f0ab..586067009fb43 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java
@@ -23,6 +23,7 @@
import com.facebook.presto.hive.metastore.SemiTransactionalHiveMetastore.WriteMode;
import com.facebook.presto.hive.metastore.StorageFormat;
import com.facebook.presto.hive.metastore.Table;
+import com.facebook.presto.hive.statistics.HiveStatisticsProvider;
import com.facebook.presto.spi.ColumnHandle;
import com.facebook.presto.spi.ColumnMetadata;
import com.facebook.presto.spi.ConnectorInsertTableHandle;
@@ -51,6 +52,7 @@
import com.facebook.presto.spi.security.GrantInfo;
import com.facebook.presto.spi.security.Privilege;
import com.facebook.presto.spi.security.PrivilegeInfo;
+import com.facebook.presto.spi.statistics.TableStatistics;
import com.facebook.presto.spi.type.Type;
import com.facebook.presto.spi.type.TypeManager;
import com.google.common.annotations.VisibleForTesting;
@@ -83,6 +85,7 @@
import java.util.function.Predicate;
import java.util.stream.Collectors;
+import static com.facebook.presto.hive.HiveBucketing.getHiveBucketHandle;
import static com.facebook.presto.hive.HiveColumnHandle.BUCKET_COLUMN_NAME;
import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.HIDDEN;
import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.PARTITION_KEY;
@@ -98,6 +101,7 @@
import static com.facebook.presto.hive.HiveErrorCode.HIVE_WRITER_CLOSE_ERROR;
import static com.facebook.presto.hive.HivePartitionManager.extractPartitionKeyValues;
import static com.facebook.presto.hive.HiveSessionProperties.isBucketExecutionEnabled;
+import static com.facebook.presto.hive.HiveSessionProperties.isStatisticsEnabled;
import static com.facebook.presto.hive.HiveTableProperties.BUCKETED_BY_PROPERTY;
import static com.facebook.presto.hive.HiveTableProperties.BUCKET_COUNT_PROPERTY;
import static com.facebook.presto.hive.HiveTableProperties.EXTERNAL_LOCATION_PROPERTY;
@@ -121,17 +125,21 @@
import static com.facebook.presto.hive.HiveWriteUtils.isWritableType;
import static com.facebook.presto.hive.metastore.HivePrivilegeInfo.toHivePrivilege;
import static com.facebook.presto.hive.metastore.MetastoreUtil.getHiveSchema;
+import static com.facebook.presto.hive.metastore.MetastoreUtil.getProtectMode;
+import static com.facebook.presto.hive.metastore.MetastoreUtil.verifyOnline;
import static com.facebook.presto.hive.metastore.PrincipalType.USER;
import static com.facebook.presto.hive.metastore.SemiTransactionalHiveMetastore.WriteMode.DIRECT_TO_TARGET_EXISTING_DIRECTORY;
import static com.facebook.presto.hive.metastore.SemiTransactionalHiveMetastore.WriteMode.DIRECT_TO_TARGET_NEW_DIRECTORY;
import static com.facebook.presto.hive.metastore.SemiTransactionalHiveMetastore.WriteMode.STAGE_AND_MOVE_TO_TARGET_DIRECTORY;
import static com.facebook.presto.hive.metastore.StorageFormat.VIEW_STORAGE_FORMAT;
import static com.facebook.presto.hive.metastore.StorageFormat.fromHiveStorageFormat;
+import static com.facebook.presto.hive.util.ConfigurationUtils.toJobConf;
import static com.facebook.presto.spi.StandardErrorCode.INVALID_SCHEMA_PROPERTY;
import static com.facebook.presto.spi.StandardErrorCode.INVALID_TABLE_PROPERTY;
import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED;
import static com.facebook.presto.spi.StandardErrorCode.SCHEMA_NOT_EMPTY;
import static com.facebook.presto.spi.predicate.TupleDomain.withColumnDomains;
+import static com.facebook.presto.spi.statistics.TableStatistics.EMPTY_STATISTICS;
import static com.google.common.base.MoreObjects.toStringHelper;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Verify.verify;
@@ -169,6 +177,7 @@ public class HiveMetadata
private final HiveStorageFormat defaultStorageFormat;
private final TypeTranslator typeTranslator;
private final String prestoVersion;
+ private final HiveStatisticsProvider hiveStatisticsProvider;
public HiveMetadata(
String connectorId,
@@ -186,7 +195,8 @@ public HiveMetadata(
TableParameterCodec tableParameterCodec,
JsonCodec partitionUpdateCodec,
TypeTranslator typeTranslator,
- String prestoVersion)
+ String prestoVersion,
+ HiveStatisticsProvider hiveStatisticsProvider)
{
this.connectorId = requireNonNull(connectorId, "connectorId is null");
@@ -206,6 +216,7 @@ public HiveMetadata(
this.defaultStorageFormat = requireNonNull(defaultStorageFormat, "defaultStorageFormat is null");
this.typeTranslator = requireNonNull(typeTranslator, "typeTranslator is null");
this.prestoVersion = requireNonNull(prestoVersion, "prestoVersion is null");
+ this.hiveStatisticsProvider = requireNonNull(hiveStatisticsProvider, "hiveStatisticsProvider is null");
}
public SemiTransactionalHiveMetastore getMetastore()
@@ -223,9 +234,11 @@ public List listSchemaNames(ConnectorSession session)
public HiveTableHandle getTableHandle(ConnectorSession session, SchemaTableName tableName)
{
requireNonNull(tableName, "tableName is null");
- if (!metastore.getTable(tableName.getSchemaName(), tableName.getTableName()).isPresent()) {
+ Optional table = metastore.getTable(tableName.getSchemaName(), tableName.getTableName());
+ if (!table.isPresent()) {
return null;
}
+ verifyOnline(tableName, Optional.empty(), getProtectMode(table.get()), table.get().getParameters());
return new HiveTableHandle(connectorId, tableName.getSchemaName(), tableName.getTableName());
}
@@ -347,6 +360,17 @@ public Map> listTableColumns(ConnectorSess
return columns.build();
}
+ @Override
+ public TableStatistics getTableStatistics(ConnectorSession session, ConnectorTableHandle tableHandle, Constraint constraint)
+ {
+ if (!isStatisticsEnabled(session)) {
+ return EMPTY_STATISTICS;
+ }
+ List hivePartitions = partitionManager.getPartitions(metastore, tableHandle, constraint).getPartitions();
+ Map tableColumns = getColumnHandles(session, tableHandle);
+ return hiveStatisticsProvider.getTableStatistics(session, tableHandle, hivePartitions, tableColumns);
+ }
+
private List listTables(ConnectorSession session, SchemaTablePrefix prefix)
{
if (prefix.getSchemaName() == null || prefix.getTableName() == null) {
@@ -715,10 +739,9 @@ private List computeFileNamesForMissingBuckets(HiveStorageFormat storage
// fast path for common case
return ImmutableList.of();
}
- JobConf conf = new JobConf(hdfsEnvironment.getConfiguration(targetPath));
+ JobConf conf = toJobConf(hdfsEnvironment.getConfiguration(targetPath));
String fileExtension = HiveWriterFactory.getFileExtension(conf, fromHiveStorageFormat(storageFormat));
- Set fileNames = partitionUpdate.getFileNames().stream()
- .collect(Collectors.toSet());
+ Set fileNames = ImmutableSet.copyOf(partitionUpdate.getFileNames());
ImmutableList.Builder missingFileNamesBuilder = ImmutableList.builder();
for (int i = 0; i < bucketCount; i++) {
String fileName = HiveWriterFactory.computeBucketedFileName(filePrefix, i) + fileExtension;
@@ -733,7 +756,7 @@ private List computeFileNamesForMissingBuckets(HiveStorageFormat storage
private void createEmptyFile(Path path, Table table, Optional partition, List fileNames)
{
- JobConf conf = new JobConf(hdfsEnvironment.getConfiguration(path));
+ JobConf conf = toJobConf(hdfsEnvironment.getConfiguration(path));
Properties schema;
StorageFormat format;
@@ -911,6 +934,8 @@ public void createView(ConnectorSession session, SchemaTableName viewName, Strin
Map properties = ImmutableMap.builder()
.put(TABLE_COMMENT, "Presto View")
.put(PRESTO_VIEW_FLAG, "true")
+ .put(PRESTO_VERSION_NAME, prestoVersion)
+ .put(PRESTO_QUERY_ID_NAME, session.getQueryId())
.build();
Column dummyColumn = new Column("dummy", HIVE_STRING, Optional.empty());
@@ -1179,21 +1204,25 @@ private static Domain buildColumnDomain(ColumnHandle column, List
@Override
public Optional getInsertLayout(ConnectorSession session, ConnectorTableHandle tableHandle)
{
- HivePartitionResult hivePartitionResult = partitionManager.getPartitions(metastore, tableHandle, Constraint.alwaysTrue());
- if (!hivePartitionResult.getBucketHandle().isPresent()) {
+ HiveTableHandle hiveTableHandle = (HiveTableHandle) tableHandle;
+ SchemaTableName tableName = hiveTableHandle.getSchemaTableName();
+ Table table = metastore.getTable(tableName.getSchemaName(), tableName.getTableName())
+ .orElseThrow(() -> new TableNotFoundException(tableName));
+
+ Optional hiveBucketHandle = getHiveBucketHandle(connectorId, table);
+ if (!hiveBucketHandle.isPresent()) {
return Optional.empty();
}
if (!bucketWritingEnabled) {
throw new PrestoException(NOT_SUPPORTED, "Writing to bucketed Hive table has been temporarily disabled");
}
- HiveBucketHandle hiveBucketHandle = hivePartitionResult.getBucketHandle().get();
HivePartitioningHandle partitioningHandle = new HivePartitioningHandle(
connectorId,
- hiveBucketHandle.getBucketCount(),
- hiveBucketHandle.getColumns().stream()
+ hiveBucketHandle.get().getBucketCount(),
+ hiveBucketHandle.get().getColumns().stream()
.map(HiveColumnHandle::getHiveType)
.collect(Collectors.toList()));
- List partitionColumns = hiveBucketHandle.getColumns().stream()
+ List partitionColumns = hiveBucketHandle.get().getColumns().stream()
.map(HiveColumnHandle::getName)
.collect(Collectors.toList());
return Optional.of(new ConnectorNewTableLayout(partitioningHandle, partitionColumns));
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadataFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadataFactory.java
index 8251b15709d64..706986fe71188 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadataFactory.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadataFactory.java
@@ -16,6 +16,7 @@
import com.facebook.presto.hive.metastore.CachingHiveMetastore;
import com.facebook.presto.hive.metastore.ExtendedHiveMetastore;
import com.facebook.presto.hive.metastore.SemiTransactionalHiveMetastore;
+import com.facebook.presto.hive.statistics.MetastoreHiveStatisticsProvider;
import com.facebook.presto.spi.type.TypeManager;
import io.airlift.concurrent.BoundedExecutor;
import io.airlift.json.JsonCodec;
@@ -145,13 +146,15 @@ public HiveMetadataFactory(
public HiveMetadata create()
{
+ SemiTransactionalHiveMetastore metastore = new SemiTransactionalHiveMetastore(
+ hdfsEnvironment,
+ CachingHiveMetastore.memoizeMetastore(this.metastore, perTransactionCacheMaximumSize), // per-transaction cache
+ renameExecution,
+ skipDeletionForAlter);
+
return new HiveMetadata(
connectorId,
- new SemiTransactionalHiveMetastore(
- hdfsEnvironment,
- CachingHiveMetastore.memoizeMetastore(metastore, perTransactionCacheMaximumSize), // per-transaction cache
- renameExecution,
- skipDeletionForAlter),
+ metastore,
hdfsEnvironment,
partitionManager,
timeZone,
@@ -165,6 +168,7 @@ public HiveMetadata create()
tableParameterCodec,
partitionUpdateCodec,
typeTranslator,
- prestoVersion);
+ prestoVersion,
+ new MetastoreHiveStatisticsProvider(typeManager, metastore));
}
}
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HivePartitionManager.java b/presto-hive/src/main/java/com/facebook/presto/hive/HivePartitionManager.java
index c1a4de0d3beb5..2fd7916cc3338 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/HivePartitionManager.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/HivePartitionManager.java
@@ -33,7 +33,6 @@
import com.google.common.collect.Maps;
import io.airlift.slice.Slice;
import org.apache.hadoop.hive.common.FileUtils;
-import org.apache.hadoop.hive.metastore.ProtectMode;
import org.joda.time.DateTimeZone;
import javax.inject.Inject;
@@ -49,14 +48,14 @@
import static com.facebook.presto.hive.HiveErrorCode.HIVE_EXCEEDED_PARTITION_LIMIT;
import static com.facebook.presto.hive.HiveUtil.getPartitionKeyColumnHandles;
import static com.facebook.presto.hive.HiveUtil.parsePartitionValue;
+import static com.facebook.presto.hive.metastore.MetastoreUtil.getProtectMode;
+import static com.facebook.presto.hive.metastore.MetastoreUtil.verifyOnline;
import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Predicates.not;
-import static com.google.common.base.Strings.isNullOrEmpty;
import static java.lang.String.format;
import static java.util.Objects.requireNonNull;
import static java.util.stream.Collectors.toList;
-import static org.apache.hadoop.hive.metastore.ProtectMode.getProtectModeFromString;
public class HivePartitionManager
{
@@ -160,20 +159,20 @@ public HivePartitionResult getPartitions(SemiTransactionalHiveMetastore metastor
private static TupleDomain toCompactTupleDomain(TupleDomain effectivePredicate, int threshold)
{
- checkArgument(effectivePredicate.getDomains().isPresent());
-
ImmutableMap.Builder builder = ImmutableMap.builder();
- for (Map.Entry entry : effectivePredicate.getDomains().get().entrySet()) {
- HiveColumnHandle hiveColumnHandle = (HiveColumnHandle) entry.getKey();
-
- ValueSet values = entry.getValue().getValues();
- ValueSet compactValueSet = values.getValuesProcessor().>transform(
- ranges -> ranges.getRangeCount() > threshold ? Optional.of(ValueSet.ofRanges(ranges.getSpan())) : Optional.empty(),
- discreteValues -> discreteValues.getValues().size() > threshold ? Optional.of(ValueSet.all(values.getType())) : Optional.empty(),
- allOrNone -> Optional.empty())
- .orElse(values);
- builder.put(hiveColumnHandle, Domain.create(compactValueSet, entry.getValue().isNullAllowed()));
- }
+ effectivePredicate.getDomains().ifPresent(domains -> {
+ for (Map.Entry entry : domains.entrySet()) {
+ HiveColumnHandle hiveColumnHandle = (HiveColumnHandle) entry.getKey();
+
+ ValueSet values = entry.getValue().getValues();
+ ValueSet compactValueSet = values.getValuesProcessor().>transform(
+ ranges -> ranges.getRangeCount() > threshold ? Optional.of(ValueSet.ofRanges(ranges.getSpan())) : Optional.empty(),
+ discreteValues -> discreteValues.getValues().size() > threshold ? Optional.of(ValueSet.all(values.getType())) : Optional.empty(),
+ allOrNone -> Optional.empty())
+ .orElse(values);
+ builder.put(hiveColumnHandle, Domain.create(compactValueSet, entry.getValue().isNullAllowed()));
+ }
+ });
return TupleDomain.withColumnDomains(builder.build());
}
@@ -209,17 +208,7 @@ private Table getTable(SemiTransactionalHiveMetastore metastore, SchemaTableName
throw new TableNotFoundException(tableName);
}
Table table = target.get();
-
- String protectMode = table.getParameters().get(ProtectMode.PARAMETER_NAME);
- if (protectMode != null && getProtectModeFromString(protectMode).offline) {
- throw new TableOfflineException(tableName, false, null);
- }
-
- String prestoOffline = table.getParameters().get(PRESTO_OFFLINE);
- if (!isNullOrEmpty(prestoOffline)) {
- throw new TableOfflineException(tableName, true, prestoOffline);
- }
-
+ verifyOnline(tableName, Optional.empty(), getProtectMode(table), table.getParameters());
return table;
}
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveS3Config.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveS3Config.java
index 3d36aa3f3fccc..d7687a0be476b 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveS3Config.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveS3Config.java
@@ -16,6 +16,7 @@
import com.google.common.base.StandardSystemProperty;
import io.airlift.configuration.Config;
import io.airlift.configuration.ConfigDescription;
+import io.airlift.configuration.ConfigSecuritySensitive;
import io.airlift.units.DataSize;
import io.airlift.units.Duration;
import io.airlift.units.MinDataSize;
@@ -73,6 +74,7 @@ public String getS3AwsSecretKey()
}
@Config("hive.s3.aws-secret-key")
+ @ConfigSecuritySensitive
public HiveS3Config setS3AwsSecretKey(String s3AwsSecretKey)
{
this.s3AwsSecretKey = s3AwsSecretKey;
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveSessionProperties.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveSessionProperties.java
index 06e12f423671f..98026bade193d 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveSessionProperties.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveSessionProperties.java
@@ -33,13 +33,14 @@ public final class HiveSessionProperties
private static final String ORC_MAX_MERGE_DISTANCE = "orc_max_merge_distance";
private static final String ORC_MAX_BUFFER_SIZE = "orc_max_buffer_size";
private static final String ORC_STREAM_BUFFER_SIZE = "orc_stream_buffer_size";
+ private static final String ORC_MAX_READ_BLOCK_SIZE = "orc_max_read_block_size";
private static final String PARQUET_PREDICATE_PUSHDOWN_ENABLED = "parquet_predicate_pushdown_enabled";
private static final String PARQUET_OPTIMIZED_READER_ENABLED = "parquet_optimized_reader_enabled";
private static final String MAX_SPLIT_SIZE = "max_split_size";
private static final String MAX_INITIAL_SPLIT_SIZE = "max_initial_split_size";
- private static final String RCFILE_OPTIMIZED_READER_ENABLED = "rcfile_optimized_reader_enabled";
- private static final String RCFILE_OPTIMIZED_WRITER_ENABLED = "rcfile_optimized_writer_enabled";
+ public static final String RCFILE_OPTIMIZED_WRITER_ENABLED = "rcfile_optimized_writer_enabled";
private static final String RCFILE_OPTIMIZED_WRITER_VALIDATE = "rcfile_optimized_writer_validate";
+ private static final String STATISTICS_ENABLED = "statistics_enabled";
private final List> sessionProperties;
@@ -77,6 +78,11 @@ public HiveSessionProperties(HiveClientConfig config)
"ORC: Size of buffer for streaming reads",
config.getOrcStreamBufferSize(),
false),
+ dataSizeSessionProperty(
+ ORC_MAX_READ_BLOCK_SIZE,
+ "ORC: Maximum size of a block to read",
+ config.getOrcMaxReadBlockSize(),
+ false),
booleanSessionProperty(
PARQUET_OPTIMIZED_READER_ENABLED,
"Experimental: Parquet: Enable optimized reader",
@@ -97,11 +103,6 @@ public HiveSessionProperties(HiveClientConfig config)
"Max initial split size",
config.getMaxInitialSplitSize(),
true),
- booleanSessionProperty(
- RCFILE_OPTIMIZED_READER_ENABLED,
- "Experimental: RCFile: Enable optimized reader",
- config.isRcfileOptimizedReaderEnabled(),
- false),
booleanSessionProperty(
RCFILE_OPTIMIZED_WRITER_ENABLED,
"Experimental: RCFile: Enable optimized writer",
@@ -110,6 +111,11 @@ public HiveSessionProperties(HiveClientConfig config)
booleanSessionProperty(
RCFILE_OPTIMIZED_WRITER_VALIDATE,
"Experimental: RCFile: Validate writer files",
+ config.isRcfileWriterValidate(),
+ false),
+ booleanSessionProperty(
+ STATISTICS_ENABLED,
+ "Experimental: Expose table statistics",
true,
false));
}
@@ -154,6 +160,11 @@ public static DataSize getOrcStreamBufferSize(ConnectorSession session)
return session.getProperty(ORC_STREAM_BUFFER_SIZE, DataSize.class);
}
+ public static DataSize getOrcMaxReadBlockSize(ConnectorSession session)
+ {
+ return session.getProperty(ORC_MAX_READ_BLOCK_SIZE, DataSize.class);
+ }
+
public static boolean isParquetPredicatePushdownEnabled(ConnectorSession session)
{
return session.getProperty(PARQUET_PREDICATE_PUSHDOWN_ENABLED, Boolean.class);
@@ -169,11 +180,6 @@ public static DataSize getMaxInitialSplitSize(ConnectorSession session)
return session.getProperty(MAX_INITIAL_SPLIT_SIZE, DataSize.class);
}
- public static boolean isRcfileOptimizedReaderEnabled(ConnectorSession session)
- {
- return session.getProperty(RCFILE_OPTIMIZED_READER_ENABLED, Boolean.class);
- }
-
public static boolean isRcfileOptimizedWriterEnabled(ConnectorSession session)
{
return session.getProperty(RCFILE_OPTIMIZED_WRITER_ENABLED, Boolean.class);
@@ -184,6 +190,11 @@ public static boolean isRcfileOptimizedWriterValidate(ConnectorSession session)
return session.getProperty(RCFILE_OPTIMIZED_WRITER_VALIDATE, Boolean.class);
}
+ public static boolean isStatisticsEnabled(ConnectorSession session)
+ {
+ return session.getProperty(STATISTICS_ENABLED, Boolean.class);
+ }
+
public static PropertyMetadata dataSizeSessionProperty(String name, String description, DataSize defaultValue, boolean hidden)
{
return new PropertyMetadata<>(
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveSplitManager.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveSplitManager.java
index d6bf69f143c80..e01095ada28f6 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveSplitManager.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveSplitManager.java
@@ -26,7 +26,6 @@
import com.facebook.presto.spi.TableNotFoundException;
import com.facebook.presto.spi.connector.ConnectorSplitManager;
import com.facebook.presto.spi.connector.ConnectorTransactionHandle;
-import com.google.common.base.Verify;
import com.google.common.collect.AbstractIterator;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
@@ -51,7 +50,6 @@
import static com.facebook.presto.hive.HiveErrorCode.HIVE_METASTORE_ERROR;
import static com.facebook.presto.hive.HiveErrorCode.HIVE_PARTITION_SCHEMA_MISMATCH;
import static com.facebook.presto.hive.HivePartition.UNPARTITIONED_ID;
-import static com.facebook.presto.hive.HiveUtil.checkCondition;
import static com.facebook.presto.hive.metastore.MetastoreUtil.makePartName;
import static com.facebook.presto.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR;
import static com.facebook.presto.spi.StandardErrorCode.SERVER_SHUTTING_DOWN;
@@ -206,12 +204,11 @@ private Iterable getPartitionMetadata(SemiTransactionalHi
ImmutableMap.Builder partitionBuilder = ImmutableMap.builder();
for (Map.Entry> entry : batch.entrySet()) {
if (!entry.getValue().isPresent()) {
- throw new PrestoException(HIVE_METASTORE_ERROR, "Partition metadata not available");
+ throw new PrestoException(HIVE_METASTORE_ERROR, "Partition no longer exists: " + entry.getKey());
}
partitionBuilder.put(entry.getKey(), entry.getValue().get());
}
Map partitions = partitionBuilder.build();
- Verify.verify(partitions.size() == partitionBatch.size());
if (partitionBatch.size() != partitions.size()) {
throw new PrestoException(GENERIC_INTERNAL_ERROR, format("Expected %s partitions but found %s", partitionBatch.size(), partitions.size()));
}
@@ -266,15 +263,25 @@ private Iterable getPartitionMetadata(SemiTransactionalHi
}
}
- Optional partitionBucketProperty = partition.getStorage().getBucketProperty();
- checkCondition(
- partitionBucketProperty.equals(bucketProperty),
- HiveErrorCode.HIVE_PARTITION_SCHEMA_MISMATCH,
- "Hive table (%s) bucketing property (%s) does not match partition (%s) bucketing property (%s)",
- hivePartition.getTableName(),
- bucketProperty,
- hivePartition.getPartitionId(),
- partitionBucketProperty);
+ if (bucketProperty.isPresent()) {
+ Optional partitionBucketProperty = partition.getStorage().getBucketProperty();
+ if (!partitionBucketProperty.isPresent()) {
+ throw new PrestoException(HIVE_PARTITION_SCHEMA_MISMATCH, format(
+ "Hive table (%s) is bucketed but partition (%s) is not bucketed",
+ hivePartition.getTableName(),
+ hivePartition.getPartitionId()));
+ }
+ if (!bucketProperty.equals(partitionBucketProperty)) {
+ throw new PrestoException(HIVE_PARTITION_SCHEMA_MISMATCH, format(
+ "Hive table (%s) bucketing (columns=%s, buckets=%s) does not match partition (%s) bucketing (columns=%s, buckets=%s)",
+ hivePartition.getTableName(),
+ bucketProperty.get().getBucketedBy(),
+ bucketProperty.get().getBucketCount(),
+ hivePartition.getPartitionId(),
+ partitionBucketProperty.get().getBucketedBy(),
+ partitionBucketProperty.get().getBucketCount()));
+ }
+ }
results.add(new HivePartitionMetadata(hivePartition, Optional.of(partition), columnCoercions.build()));
}
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveUtil.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveUtil.java
index 7feff178cee51..f6c7ab02b9177 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveUtil.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveUtil.java
@@ -28,8 +28,11 @@
import com.facebook.presto.spi.type.Type;
import com.facebook.presto.spi.type.VarcharType;
import com.google.common.base.Joiner;
+import com.google.common.base.Splitter;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
+import io.airlift.compress.lzo.LzoCodec;
+import io.airlift.compress.lzo.LzopCodec;
import io.airlift.slice.Slice;
import io.airlift.slice.SliceUtf8;
import io.airlift.slice.Slices;
@@ -89,11 +92,12 @@
import static com.facebook.presto.hive.HivePartitionKey.HIVE_DEFAULT_DYNAMIC_PARTITION;
import static com.facebook.presto.hive.RetryDriver.retry;
import static com.facebook.presto.hive.metastore.MetastoreUtil.getHiveSchema;
+import static com.facebook.presto.hive.util.ConfigurationUtils.toJobConf;
import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED;
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.Chars.isCharType;
-import static com.facebook.presto.spi.type.Chars.trimSpaces;
+import static com.facebook.presto.spi.type.Chars.trimTrailingSpaces;
import static com.facebook.presto.spi.type.DateType.DATE;
import static com.facebook.presto.spi.type.DecimalType.createDecimalType;
import static com.facebook.presto.spi.type.DoubleType.DOUBLE;
@@ -104,6 +108,7 @@
import static com.facebook.presto.spi.type.TinyintType.TINYINT;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.collect.Iterables.filter;
+import static com.google.common.collect.Lists.newArrayList;
import static com.google.common.collect.Lists.transform;
import static java.lang.Byte.parseByte;
import static java.lang.Double.parseDouble;
@@ -116,6 +121,7 @@
import static java.math.BigDecimal.ROUND_UNNECESSARY;
import static java.nio.charset.StandardCharsets.UTF_8;
import static java.util.Objects.requireNonNull;
+import static java.util.stream.Collectors.joining;
import static org.apache.hadoop.hive.common.FileUtils.unescapePathName;
import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.FILE_INPUT_FORMAT;
import static org.apache.hadoop.hive.serde.serdeConstants.DECIMAL_TYPE_NAME;
@@ -167,7 +173,7 @@ private HiveUtil()
setReadColumns(configuration, readHiveColumnIndexes);
InputFormat, ?> inputFormat = getInputFormat(configuration, schema, true);
- JobConf jobConf = new JobConf(configuration);
+ JobConf jobConf = toJobConf(configuration);
FileSplit fileSplit = new FileSplit(path, start, length, (String[]) null);
// propagate serialization configuration to getRecordReader
@@ -175,6 +181,16 @@ private HiveUtil()
.filter(name -> name.startsWith("serialization."))
.forEach(name -> jobConf.set(name, schema.getProperty(name)));
+ // add Airlift LZO and LZOP to head of codecs list so as to not override existing entries
+ List codecs = newArrayList(Splitter.on(",").trimResults().omitEmptyStrings().split(jobConf.get("io.compression.codecs", "")));
+ if (!codecs.contains(LzoCodec.class.getName())) {
+ codecs.add(0, LzoCodec.class.getName());
+ }
+ if (!codecs.contains(LzopCodec.class.getName())) {
+ codecs.add(0, LzopCodec.class.getName());
+ }
+ jobConf.set("io.compression.codecs", codecs.stream().collect(joining(",")));
+
try {
return retry()
.stopOnIllegalExceptions()
@@ -201,7 +217,7 @@ public static void setReadColumns(Configuration configuration, List rea
{
String inputFormatName = getInputFormatName(schema);
try {
- JobConf jobConf = new JobConf(configuration);
+ JobConf jobConf = toJobConf(configuration);
Class extends InputFormat, ?>> inputFormatClass = getInputFormatClass(jobConf, inputFormatName);
if (symlinkTarget && (inputFormatClass == SymlinkTextInputFormat.class)) {
@@ -688,7 +704,7 @@ public static Slice varcharPartitionKey(String value, String name, Type columnTy
public static Slice charPartitionKey(String value, String name, Type columnType)
{
- Slice partitionKey = trimSpaces(Slices.utf8Slice(value));
+ Slice partitionKey = trimTrailingSpaces(Slices.utf8Slice(value));
CharType charType = (CharType) columnType;
if (SliceUtf8.countCodePoints(partitionKey) > charType.getLength()) {
throw new PrestoException(HIVE_INVALID_PARTITION_VALUE, format("Invalid partition value '%s' for %s partition key: %s", value, columnType.toString(), name));
@@ -753,11 +769,6 @@ public static List getPartitionKeyColumnHandles(String connect
return columns.build();
}
- public static Slice base64Decode(byte[] bytes)
- {
- return Slices.wrappedBuffer(Base64.getDecoder().decode(bytes));
- }
-
public static void checkCondition(boolean condition, ErrorCodeSupplier errorCode, String formatString, Object... args)
{
if (!condition) {
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java
index ee7dd77ed1332..4b5dc72867c75 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java
@@ -96,15 +96,14 @@
import static com.facebook.presto.hive.HiveErrorCode.HIVE_DATABASE_LOCATION_ERROR;
import static com.facebook.presto.hive.HiveErrorCode.HIVE_FILESYSTEM_ERROR;
import static com.facebook.presto.hive.HiveErrorCode.HIVE_WRITER_DATA_ERROR;
-import static com.facebook.presto.hive.HiveSplitManager.PRESTO_OFFLINE;
import static com.facebook.presto.hive.HiveUtil.checkCondition;
import static com.facebook.presto.hive.HiveUtil.isArrayType;
import static com.facebook.presto.hive.HiveUtil.isMapType;
import static com.facebook.presto.hive.HiveUtil.isRowType;
import static com.facebook.presto.hive.metastore.MetastoreUtil.getProtectMode;
+import static com.facebook.presto.hive.metastore.MetastoreUtil.verifyOnline;
import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED;
import static com.facebook.presto.spi.type.Chars.isCharType;
-import static com.google.common.base.Strings.isNullOrEmpty;
import static com.google.common.base.Strings.padEnd;
import static java.lang.Float.intBitsToFloat;
import static java.lang.Math.toIntExact;
@@ -367,20 +366,7 @@ private static void checkWritable(
}
// verify online
- if (protectMode.offline) {
- if (partitionName.isPresent()) {
- throw new PartitionOfflineException(tableName, partitionName.get(), false, null);
- }
- throw new TableOfflineException(tableName, false, null);
- }
-
- String prestoOffline = parameters.get(PRESTO_OFFLINE);
- if (!isNullOrEmpty(prestoOffline)) {
- if (partitionName.isPresent()) {
- throw new PartitionOfflineException(tableName, partitionName.get(), true, prestoOffline);
- }
- throw new TableOfflineException(tableName, true, prestoOffline);
- }
+ verifyOnline(tableName, partitionName, protectMode, parameters);
// verify not read only
if (protectMode.readOnly) {
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriterFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriterFactory.java
index 910ea7333e940..8e60fc63a6ea0 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriterFactory.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriterFactory.java
@@ -63,6 +63,7 @@
import static com.facebook.presto.hive.HiveWriteUtils.getField;
import static com.facebook.presto.hive.metastore.MetastoreUtil.getHiveSchema;
import static com.facebook.presto.hive.metastore.StorageFormat.fromHiveStorageFormat;
+import static com.facebook.presto.hive.util.ConfigurationUtils.toJobConf;
import static com.facebook.presto.spi.StandardErrorCode.NOT_FOUND;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.collect.ImmutableMap.toImmutableMap;
@@ -201,7 +202,7 @@ public HiveWriterFactory(
entry -> session.getProperty(entry.getName(), entry.getJavaType()).toString()));
Configuration conf = hdfsEnvironment.getConfiguration(writePath);
- this.conf = new JobConf(conf);
+ this.conf = toJobConf(conf);
// make sure the FileSystem is created with the correct Configuration object
try {
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/NumberParser.java b/presto-hive/src/main/java/com/facebook/presto/hive/NumberParser.java
deleted file mode 100644
index 9c359bf8f4ec6..0000000000000
--- a/presto-hive/src/main/java/com/facebook/presto/hive/NumberParser.java
+++ /dev/null
@@ -1,65 +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.hive;
-
-import com.facebook.presto.spi.PrestoException;
-
-import static com.facebook.presto.hive.HiveErrorCode.HIVE_BAD_DATA;
-
-public final class NumberParser
-{
- private NumberParser() {}
-
- public static long parseLong(byte[] bytes, int start, int length)
- {
- int limit = start + length;
-
- int sign = bytes[start] == '-' ? -1 : 1;
-
- if (sign == -1 || bytes[start] == '+') {
- start++;
- }
-
- long value = bytes[start] - ((int) '0');
- start++;
- while (start < limit) {
- value = value * 10 + (bytes[start] - ((int) '0'));
- start++;
- }
-
- return value * sign;
- }
-
- public static float parseFloat(byte[] bytes, int start, int length)
- {
- String string = new String(bytes, 0, start, length);
- try {
- return Float.parseFloat(string);
- }
- catch (NumberFormatException e) {
- throw new PrestoException(HIVE_BAD_DATA, e);
- }
- }
-
- public static double parseDouble(byte[] bytes, int start, int length)
- {
- String string = new String(bytes, 0, start, length);
- try {
- return Double.parseDouble(string);
- }
- catch (NumberFormatException e) {
- throw new PrestoException(HIVE_BAD_DATA, e);
- }
- }
-}
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/PartitionStatistics.java b/presto-hive/src/main/java/com/facebook/presto/hive/PartitionStatistics.java
new file mode 100644
index 0000000000000..bfb7ec9ff698c
--- /dev/null
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/PartitionStatistics.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.facebook.presto.hive;
+
+import com.facebook.presto.hive.metastore.HiveColumnStatistics;
+import com.google.common.collect.ImmutableMap;
+
+import java.util.Map;
+import java.util.OptionalLong;
+
+import static java.util.Objects.requireNonNull;
+
+public class PartitionStatistics
+{
+ public static final PartitionStatistics EMPTY_STATISTICS = new PartitionStatistics(
+ false,
+ OptionalLong.empty(),
+ OptionalLong.empty(),
+ OptionalLong.empty(),
+ OptionalLong.empty(),
+ ImmutableMap.of());
+
+ private final boolean columnStatsAcurate;
+ private final OptionalLong fileCount;
+ private final OptionalLong rowCount;
+ private final OptionalLong rawDataSize;
+ private final OptionalLong totalSize;
+ private final Map columnStatistics;
+
+ public PartitionStatistics(
+ boolean columnStatsAcurate,
+ OptionalLong fileCount,
+ OptionalLong rowCount,
+ OptionalLong rawDataSize,
+ OptionalLong totalSize,
+ Map columnStatistics)
+ {
+ this.columnStatsAcurate = columnStatsAcurate;
+ this.fileCount = fileCount;
+ this.rowCount = rowCount;
+ this.rawDataSize = rawDataSize;
+ this.totalSize = totalSize;
+ this.columnStatistics = ImmutableMap.copyOf(requireNonNull(columnStatistics, "columnStatistics can not be null"));
+ }
+
+ public boolean isColumnStatsAcurate()
+ {
+ return columnStatsAcurate;
+ }
+
+ public OptionalLong getFileCount()
+ {
+ return fileCount;
+ }
+
+ public OptionalLong getRowCount()
+ {
+ return rowCount;
+ }
+
+ public OptionalLong getRawDataSize()
+ {
+ return rawDataSize;
+ }
+
+ public OptionalLong getTotalSize()
+ {
+ return totalSize;
+ }
+
+ public Map getColumnStatistics()
+ {
+ return columnStatistics;
+ }
+}
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/PrestoHadoopConfiguration.java b/presto-hive/src/main/java/com/facebook/presto/hive/PrestoHadoopConfiguration.java
deleted file mode 100644
index f65d619b34c7b..0000000000000
--- a/presto-hive/src/main/java/com/facebook/presto/hive/PrestoHadoopConfiguration.java
+++ /dev/null
@@ -1,40 +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.hive;
-
-import com.google.common.collect.ClassToInstanceMap;
-import com.google.common.collect.ImmutableClassToInstanceMap;
-import org.apache.hadoop.conf.Configuration;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static java.util.Objects.requireNonNull;
-
-public final class PrestoHadoopConfiguration
- extends Configuration
-{
- private final ClassToInstanceMap services;
-
- public PrestoHadoopConfiguration(ClassToInstanceMap> services)
- {
- super(false);
- this.services = ImmutableClassToInstanceMap.copyOf(requireNonNull(services, "services is null"));
- }
-
- public T getService(Class type)
- {
- T service = services.getInstance(type);
- checkArgument(service != null, "service not found: %s", type.getName());
- return service;
- }
-}
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/PrestoS3FileSystem.java b/presto-hive/src/main/java/com/facebook/presto/hive/PrestoS3FileSystem.java
index a36e2f5afa99d..dd252ef757466 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/PrestoS3FileSystem.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/PrestoS3FileSystem.java
@@ -83,8 +83,10 @@
import java.util.Date;
import java.util.Iterator;
import java.util.List;
+import java.util.Map;
import java.util.Optional;
+import static com.amazonaws.services.s3.Headers.SERVER_SIDE_ENCRYPTION;
import static com.amazonaws.services.s3.Headers.UNENCRYPTED_CONTENT_LENGTH;
import static com.facebook.presto.hive.RetryDriver.retry;
import static com.google.common.base.Preconditions.checkArgument;
@@ -315,7 +317,7 @@ public FileStatus getFileStatus(Path path)
}
return new FileStatus(
- getObjectSize(metadata),
+ getObjectSize(path, metadata),
false,
1,
BLOCK_SIZE.toBytes(),
@@ -323,9 +325,14 @@ public FileStatus getFileStatus(Path path)
qualifiedPath(path));
}
- private static long getObjectSize(ObjectMetadata metadata)
+ private static long getObjectSize(Path path, ObjectMetadata metadata)
+ throws IOException
{
- String length = metadata.getUserMetadata().get(UNENCRYPTED_CONTENT_LENGTH);
+ Map userMetadata = metadata.getUserMetadata();
+ String length = userMetadata.get(UNENCRYPTED_CONTENT_LENGTH);
+ if (userMetadata.containsKey(SERVER_SIDE_ENCRYPTION) && length == null) {
+ throw new IOException(format("%s header is not set on an encrypted object: %s", UNENCRYPTED_CONTENT_LENGTH, path));
+ }
return (length != null) ? Long.parseLong(length) : metadata.getContentLength();
}
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/RcFileFileWriterFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/RcFileFileWriterFactory.java
index e56f17508fd31..8bbc267320401 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/RcFileFileWriterFactory.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/RcFileFileWriterFactory.java
@@ -57,27 +57,31 @@ public class RcFileFileWriterFactory
private final HdfsEnvironment hdfsEnvironment;
private final TypeManager typeManager;
private final NodeVersion nodeVersion;
+ private final FileFormatDataSourceStats stats;
@Inject
public RcFileFileWriterFactory(
HdfsEnvironment hdfsEnvironment,
TypeManager typeManager,
NodeVersion nodeVersion,
- HiveClientConfig hiveClientConfig)
+ HiveClientConfig hiveClientConfig,
+ FileFormatDataSourceStats stats)
{
- this(hdfsEnvironment, typeManager, nodeVersion, requireNonNull(hiveClientConfig, "hiveClientConfig is null").getDateTimeZone());
+ this(hdfsEnvironment, typeManager, nodeVersion, requireNonNull(hiveClientConfig, "hiveClientConfig is null").getDateTimeZone(), stats);
}
public RcFileFileWriterFactory(
HdfsEnvironment hdfsEnvironment,
TypeManager typeManager,
NodeVersion nodeVersion,
- DateTimeZone hiveStorageTimeZone)
+ DateTimeZone hiveStorageTimeZone,
+ FileFormatDataSourceStats stats)
{
this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null");
this.typeManager = requireNonNull(typeManager, "typeManager is null");
this.nodeVersion = requireNonNull(nodeVersion, "nodeVersion is null");
this.hiveStorageTimeZone = requireNonNull(hiveStorageTimeZone, "hiveStorageTimeZone is null");
+ this.stats = requireNonNull(stats, "stats is null");
}
@Override
@@ -132,7 +136,8 @@ else if (ColumnarSerDe.class.getName().equals(storageFormat.getSerDe())) {
return new HdfsRcFileDataSource(
path.toString(),
fileSystem.open(path),
- fileSystem.getFileStatus(path).getLen());
+ fileSystem.getFileStatus(path).getLen(),
+ stats);
}
catch (IOException e) {
throw Throwables.propagate(e);
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/StaticHiveCluster.java b/presto-hive/src/main/java/com/facebook/presto/hive/StaticHiveCluster.java
index 3a37c831cbe46..4facd9771f38a 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/StaticHiveCluster.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/StaticHiveCluster.java
@@ -71,7 +71,7 @@ public HiveMetastoreClient createMetastoreClient()
TTransportException lastException = null;
for (HostAndPort metastore : metastores) {
try {
- return clientFactory.create(metastore.getHostText(), metastore.getPort());
+ return clientFactory.create(metastore.getHost(), metastore.getPort());
}
catch (TTransportException e) {
lastException = e;
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/ThriftHiveMetastoreClient.java b/presto-hive/src/main/java/com/facebook/presto/hive/ThriftHiveMetastoreClient.java
index b037f40b75a7a..4208acc3c8627 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/ThriftHiveMetastoreClient.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/ThriftHiveMetastoreClient.java
@@ -14,15 +14,18 @@
package com.facebook.presto.hive;
import com.facebook.presto.hive.metastore.HiveMetastoreClient;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
import org.apache.hadoop.hive.metastore.api.Database;
import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.PartitionsStatsRequest;
import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
import org.apache.hadoop.hive.metastore.api.PrincipalType;
import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
import org.apache.hadoop.hive.metastore.api.Role;
import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TableStatsRequest;
import org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore;
import org.apache.thrift.TException;
import org.apache.thrift.protocol.TBinaryProtocol;
@@ -30,6 +33,7 @@
import org.apache.thrift.transport.TTransport;
import java.util.List;
+import java.util.Map;
import static java.util.Objects.requireNonNull;
@@ -134,6 +138,22 @@ public Table getTable(String databaseName, String tableName)
return client.get_table(databaseName, tableName);
}
+ @Override
+ public List getTableColumnStatistics(String databaseName, String tableName, List columnNames)
+ throws TException
+ {
+ TableStatsRequest tableStatsRequest = new TableStatsRequest(databaseName, tableName, columnNames);
+ return client.get_table_statistics_req(tableStatsRequest).getTableStats();
+ }
+
+ @Override
+ public Map> getPartitionColumnStatistics(String databaseName, String tableName, List columnNames, List partitionValues)
+ throws TException
+ {
+ PartitionsStatsRequest partitionsStatsRequest = new PartitionsStatsRequest(databaseName, tableName, columnNames, partitionValues);
+ return client.get_partitions_statistics_req(partitionsStatsRequest).getPartStats();
+ }
+
@Override
public List getPartitionNames(String databaseName, String tableName)
throws TException
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/WriteCompletedEvent.java b/presto-hive/src/main/java/com/facebook/presto/hive/WriteCompletedEvent.java
index 10288aa8629eb..c1b3591da07ca 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/WriteCompletedEvent.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/WriteCompletedEvent.java
@@ -14,11 +14,13 @@
package com.facebook.presto.hive;
import io.airlift.event.client.EventField;
+import io.airlift.event.client.EventField.EventFieldMapping;
import io.airlift.event.client.EventType;
import javax.annotation.Nullable;
import javax.annotation.concurrent.Immutable;
+import java.time.Instant;
import java.util.Map;
import static java.util.Objects.requireNonNull;
@@ -35,12 +37,13 @@ public class WriteCompletedEvent
private final String storageFormat;
private final String writerImplementation;
private final String prestoVersion;
- private final String serverAddress;
+ private final String host;
private final String principal;
private final String environment;
private final Map sessionProperties;
private final Long bytes;
private final long rows;
+ private final Instant timestamp = Instant.now();
public WriteCompletedEvent(
String queryId,
@@ -66,7 +69,7 @@ public WriteCompletedEvent(
this.storageFormat = requireNonNull(storageFormat, "storageFormat is null");
this.writerImplementation = requireNonNull(writerImplementation, "writerImplementation is null");
this.prestoVersion = requireNonNull(prestoVersion, "prestoVersion is null");
- this.serverAddress = requireNonNull(serverAddress, "serverAddress is null");
+ this.host = requireNonNull(serverAddress, "serverAddress is null");
this.principal = principal;
this.environment = requireNonNull(environment, "environment is null");
this.sessionProperties = requireNonNull(sessionProperties, "sessionProperties is null");
@@ -123,10 +126,10 @@ public String getPrestoVersion()
return prestoVersion;
}
- @EventField
- public String getServerAddress()
+ @EventField(fieldMapping = EventFieldMapping.HOST)
+ public String getHost()
{
- return serverAddress;
+ return host;
}
@Nullable
@@ -160,4 +163,10 @@ public long getRows()
{
return rows;
}
+
+ @EventField(fieldMapping = EventFieldMapping.TIMESTAMP)
+ public Instant getTimestamp()
+ {
+ return timestamp;
+ }
}
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/authentication/AuthenticationModules.java b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/AuthenticationModules.java
index 285bf293cfa71..0059c118b6d54 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/authentication/AuthenticationModules.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/AuthenticationModules.java
@@ -15,7 +15,6 @@
import com.facebook.presto.hive.ForHdfs;
import com.facebook.presto.hive.ForHiveMetastore;
-import com.facebook.presto.hive.HiveClientConfig;
import com.google.inject.Binder;
import com.google.inject.Key;
import com.google.inject.Module;
@@ -25,6 +24,7 @@
import javax.inject.Inject;
import static com.google.inject.Scopes.SINGLETON;
+import static io.airlift.configuration.ConfigBinder.configBinder;
public final class AuthenticationModules
{
@@ -48,15 +48,16 @@ public void configure(Binder binder)
binder.bind(HiveMetastoreAuthentication.class)
.to(KerberosHiveMetastoreAuthentication.class)
.in(SINGLETON);
+ configBinder(binder).bindConfig(MetastoreKerberosConfig.class);
}
@Provides
@Singleton
@ForHiveMetastore
- HadoopAuthentication createHadoopAuthentication(HiveClientConfig hiveClientConfig)
+ HadoopAuthentication createHadoopAuthentication(MetastoreKerberosConfig config)
{
- String principal = hiveClientConfig.getHiveMetastoreClientPrincipal();
- String keytabLocation = hiveClientConfig.getHiveMetastoreClientKeytab();
+ String principal = config.getHiveMetastoreClientPrincipal();
+ String keytabLocation = config.getHiveMetastoreClientKeytab();
return createCachingKerberosHadoopAuthentication(principal, keytabLocation);
}
};
@@ -91,16 +92,17 @@ public void configure(Binder binder)
binder.bind(HdfsAuthentication.class)
.to(DirectHdfsAuthentication.class)
.in(SINGLETON);
+ configBinder(binder).bindConfig(HdfsKerberosConfig.class);
}
@Inject
@Provides
@Singleton
@ForHdfs
- HadoopAuthentication createHadoopAuthentication(HiveClientConfig hiveClientConfig)
+ HadoopAuthentication createHadoopAuthentication(HdfsKerberosConfig config)
{
- String principal = hiveClientConfig.getHdfsPrestoPrincipal();
- String keytabLocation = hiveClientConfig.getHdfsPrestoKeytab();
+ String principal = config.getHdfsPrestoPrincipal();
+ String keytabLocation = config.getHdfsPrestoKeytab();
return createCachingKerberosHadoopAuthentication(principal, keytabLocation);
}
};
@@ -116,16 +118,17 @@ public void configure(Binder binder)
binder.bind(HdfsAuthentication.class)
.to(ImpersonatingHdfsAuthentication.class)
.in(SINGLETON);
+ configBinder(binder).bindConfig(HdfsKerberosConfig.class);
}
@Inject
@Provides
@Singleton
@ForHdfs
- HadoopAuthentication createHadoopAuthentication(HiveClientConfig hiveClientConfig)
+ HadoopAuthentication createHadoopAuthentication(HdfsKerberosConfig config)
{
- String principal = hiveClientConfig.getHdfsPrestoPrincipal();
- String keytabLocation = hiveClientConfig.getHdfsPrestoKeytab();
+ String principal = config.getHdfsPrestoPrincipal();
+ String keytabLocation = config.getHdfsPrestoKeytab();
return createCachingKerberosHadoopAuthentication(principal, keytabLocation);
}
};
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/authentication/HdfsKerberosConfig.java b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/HdfsKerberosConfig.java
new file mode 100644
index 0000000000000..23938da6a6bc4
--- /dev/null
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/HdfsKerberosConfig.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.facebook.presto.hive.authentication;
+
+import io.airlift.configuration.Config;
+import io.airlift.configuration.ConfigDescription;
+
+import javax.validation.constraints.NotNull;
+
+public class HdfsKerberosConfig
+{
+ private String hdfsPrestoPrincipal;
+ private String hdfsPrestoKeytab;
+
+ @NotNull
+ public String getHdfsPrestoPrincipal()
+ {
+ return hdfsPrestoPrincipal;
+ }
+
+ @Config("hive.hdfs.presto.principal")
+ @ConfigDescription("Presto principal used to access HDFS")
+ public HdfsKerberosConfig setHdfsPrestoPrincipal(String hdfsPrestoPrincipal)
+ {
+ this.hdfsPrestoPrincipal = hdfsPrestoPrincipal;
+ return this;
+ }
+
+ @NotNull
+ public String getHdfsPrestoKeytab()
+ {
+ return hdfsPrestoKeytab;
+ }
+
+ @Config("hive.hdfs.presto.keytab")
+ @ConfigDescription("Presto keytab used to access HDFS")
+ public HdfsKerberosConfig setHdfsPrestoKeytab(String hdfsPrestoKeytab)
+ {
+ this.hdfsPrestoKeytab = hdfsPrestoKeytab;
+ return this;
+ }
+}
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/authentication/KerberosHiveMetastoreAuthentication.java b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/KerberosHiveMetastoreAuthentication.java
index 41d4d79326fde..4b15870f57825 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/authentication/KerberosHiveMetastoreAuthentication.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/KerberosHiveMetastoreAuthentication.java
@@ -14,7 +14,6 @@
package com.facebook.presto.hive.authentication;
import com.facebook.presto.hive.ForHiveMetastore;
-import com.facebook.presto.hive.HiveClientConfig;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import org.apache.hadoop.hive.thrift.client.TUGIAssumingTransport;
@@ -41,9 +40,9 @@ public class KerberosHiveMetastoreAuthentication
private final HadoopAuthentication authentication;
@Inject
- public KerberosHiveMetastoreAuthentication(HiveClientConfig hiveClientConfig, @ForHiveMetastore HadoopAuthentication authentication)
+ public KerberosHiveMetastoreAuthentication(MetastoreKerberosConfig config, @ForHiveMetastore HadoopAuthentication authentication)
{
- this(hiveClientConfig.getHiveMetastoreServicePrincipal(), authentication);
+ this(config.getHiveMetastoreServicePrincipal(), authentication);
}
public KerberosHiveMetastoreAuthentication(String hiveMetastoreServicePrincipal, HadoopAuthentication authentication)
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/authentication/MetastoreKerberosConfig.java b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/MetastoreKerberosConfig.java
new file mode 100644
index 0000000000000..33b64a36966d0
--- /dev/null
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/MetastoreKerberosConfig.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.facebook.presto.hive.authentication;
+
+import io.airlift.configuration.Config;
+import io.airlift.configuration.ConfigDescription;
+
+import javax.validation.constraints.NotNull;
+
+public class MetastoreKerberosConfig
+{
+ private String hiveMetastoreServicePrincipal;
+ private String hiveMetastoreClientPrincipal;
+ private String hiveMetastoreClientKeytab;
+
+ @NotNull
+ public String getHiveMetastoreServicePrincipal()
+ {
+ return hiveMetastoreServicePrincipal;
+ }
+
+ @Config("hive.metastore.service.principal")
+ @ConfigDescription("Hive Metastore service principal")
+ public MetastoreKerberosConfig setHiveMetastoreServicePrincipal(String hiveMetastoreServicePrincipal)
+ {
+ this.hiveMetastoreServicePrincipal = hiveMetastoreServicePrincipal;
+ return this;
+ }
+
+ @NotNull
+ public String getHiveMetastoreClientPrincipal()
+ {
+ return hiveMetastoreClientPrincipal;
+ }
+
+ @Config("hive.metastore.client.principal")
+ @ConfigDescription("Hive Metastore client principal")
+ public MetastoreKerberosConfig setHiveMetastoreClientPrincipal(String hiveMetastoreClientPrincipal)
+ {
+ this.hiveMetastoreClientPrincipal = hiveMetastoreClientPrincipal;
+ return this;
+ }
+
+ @NotNull
+ public String getHiveMetastoreClientKeytab()
+ {
+ return hiveMetastoreClientKeytab;
+ }
+
+ @Config("hive.metastore.client.keytab")
+ @ConfigDescription("Hive Metastore client keytab location")
+ public MetastoreKerberosConfig setHiveMetastoreClientKeytab(String hiveMetastoreClientKeytab)
+ {
+ this.hiveMetastoreClientKeytab = hiveMetastoreClientKeytab;
+ return this;
+ }
+}
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/BridgingHiveMetastore.java b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/BridgingHiveMetastore.java
index 7caed88eabe68..c43c2f0b85bb0 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/BridgingHiveMetastore.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/BridgingHiveMetastore.java
@@ -20,6 +20,7 @@
import com.facebook.presto.spi.SchemaTableName;
import com.facebook.presto.spi.TableNotFoundException;
import com.google.common.collect.ImmutableMap;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
import org.apache.hadoop.hive.metastore.api.FieldSchema;
import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo;
@@ -73,6 +74,33 @@ public Optional getTable(String databaseName, String tableName)
return delegate.getTable(databaseName, tableName).map(MetastoreUtil::fromMetastoreApiTable);
}
+ @Override
+ public Optional