diff --git a/.travis.yml b/.travis.yml
index 9b48214fd8ec..227521b43994 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -5,7 +5,8 @@ env:
- MAVEN_OPTS="-Xmx512M -XX:+ExitOnOutOfMemoryError"
- MAVEN_SKIP_CHECKS_AND_DOCS="-Dair.check.skip-all=true -Dmaven.javadoc.skip=true"
- MAVEN_FAST_INSTALL="-DskipTests $MAVEN_SKIP_CHECKS_AND_DOCS -B -q -T C1"
- - ARTIFACTS_UPLOAD_PATH=travis_build_artifacts/${TRAVIS_REPO_SLUG}/${TRAVIS_BRANCH}/${TRAVIS_BUILD_NUMBER}
+ - ARTIFACTS_UPLOAD_PATH_BRANCH=travis_build_artifacts/${TRAVIS_REPO_SLUG}/${TRAVIS_BRANCH}/${TRAVIS_BUILD_NUMBER}
+ - ARTIFACTS_UPLOAD_PATH_PR=travis_build_artifacts_pr/${TRAVIS_REPO_SLUG}/${TRAVIS_BRANCH}/${TRAVIS_BUILD_NUMBER}
- TEST_FLAGS=""
matrix:
- MAVEN_CHECKS=true
@@ -16,11 +17,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 +47,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
- |
@@ -48,6 +55,13 @@ install:
./mvnw install $MAVEN_FAST_INSTALL -pl presto-hive-hadoop2 -am
fi
+before_script:
+ - |
+ export ARTIFACTS_UPLOAD_PATH=${ARTIFACTS_UPLOAD_PATH_BRANCH}
+ if [ "$TRAVIS_PULL_REQUEST" != "false" ]; then
+ export ARTIFACTS_UPLOAD_PATH=${ARTIFACTS_UPLOAD_PATH_PR}
+ fi
+
script:
- |
if [[ -v MAVEN_CHECKS ]]; then
@@ -62,28 +76,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 7df5380c68c4..f7428dba837d 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
@@ -83,3 +83,4 @@ We recommend you use IntelliJ as your IDE. The code style template for the proje
* Consider using String formatting (printf style formatting using the Java `Formatter` class): `format("Session property %s is invalid: %s", name, value)` (note that `format()` should always be statically imported). Sometimes, if you only need to append something, consider using the `+` operator.
* Avoid using the ternary operator except for trivial expressions.
* Use an assertion from Airlift's `Assertions` class if there is one that covers your case rather than writing the assertion by hand. Over time we may move over to more fluent assertions like AssertJ.
+* When writing a Git commit message, follow these [guidelines](https://chris.beams.io/posts/git-commit/).
diff --git a/pom.xml b/pom.xml
index 35ab6862c142..ddaf55607f08 100644
--- a/pom.xml
+++ b/pom.xml
@@ -10,7 +10,7 @@
com.facebook.presto
presto-root
- 0.179-tw-0.36
+ 0.181-tw-0.37
pom
presto-root
@@ -30,7 +30,7 @@
scm:git:git://github.com/twitter-forks/presto.git
https://github.com/twitter-forks/presto
- 0.179-tw-0.36
+ 0.181-tw-0.37
@@ -51,11 +51,11 @@
${dep.airlift.version}
0.29
1.11.30
+ 3.8.1
1.31
6.10
-
- true
- None
+ 0.15.1
+ 0.15.2
Asia/Katmandu
@@ -112,6 +112,9 @@
presto-plugin-toolkit
presto-resource-group-managers
presto-benchto-benchmarks
+ presto-thrift-connector-api
+ presto-thrift-testing-server
+ presto-thrift-connector
@@ -344,6 +347,32 @@
3
+
+ com.facebook.presto
+ presto-thrift-connector-api
+ ${project.version}
+
+
+
+ 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
+
+
com.facebook.hive
hive-dwrf
@@ -353,7 +382,7 @@
io.airlift
aircompressor
- 0.7
+ 0.8
@@ -604,6 +633,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
@@ -693,7 +770,7 @@
io.airlift
testing-postgresql-server
- 9.6.1-1
+ 9.6.3-1
@@ -890,7 +967,7 @@
org.codehaus.mojo
exec-maven-plugin
- 1.2.1
+ 1.6.0
@@ -912,6 +989,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
+
+
+
+
+
@@ -1087,26 +1185,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 a429b78ed102..c7b0cb709357 100644
--- a/presto-accumulo/pom.xml
+++ b/presto-accumulo/pom.xml
@@ -5,7 +5,7 @@
com.facebook.presto
presto-root
- 0.179-tw-0.36
+ 0.181-tw-0.37
presto-accumulo
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 8c0ee86ab263..f9edfe46d5cf 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
@@ -22,6 +22,7 @@
import com.google.common.collect.ImmutableListMultimap;
import com.google.common.collect.ImmutableMultimap;
import com.google.common.collect.Multimap;
+import io.airlift.concurrent.BoundedExecutor;
import io.airlift.log.Logger;
import io.airlift.units.Duration;
import org.apache.accumulo.core.client.AccumuloException;
@@ -36,17 +37,25 @@
import org.apache.accumulo.core.security.Authorizations;
import org.apache.hadoop.io.Text;
+import javax.annotation.PreDestroy;
import javax.inject.Inject;
+import java.util.ArrayList;
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.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import static com.facebook.presto.accumulo.AccumuloClient.getRangesFromDomain;
+import static com.facebook.presto.accumulo.AccumuloErrorCode.UNEXPECTED_ACCUMULO_ERROR;
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;
@@ -57,13 +66,13 @@
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;
+import static io.airlift.concurrent.Threads.daemonThreadsNamed;
import static java.util.Objects.requireNonNull;
+import static java.util.concurrent.Executors.newCachedThreadPool;
/**
* Class to assist the Presto connector, and maybe external applications,
@@ -77,12 +86,24 @@ public class IndexLookup
private static final Range METRICS_TABLE_ROWID_RANGE = new Range(METRICS_TABLE_ROWID_AS_TEXT);
private final ColumnCardinalityCache cardinalityCache;
private final Connector connector;
+ private final ExecutorService coreExecutor;
+ private final BoundedExecutor executorService;
@Inject
public IndexLookup(Connector connector, ColumnCardinalityCache cardinalityCache)
{
this.connector = requireNonNull(connector, "connector is null");
this.cardinalityCache = requireNonNull(cardinalityCache, "cardinalityCache is null");
+
+ // 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());
+ }
+
+ @PreDestroy
+ public void shutdown()
+ {
+ coreExecutor.shutdownNow();
}
/**
@@ -222,7 +243,7 @@ private boolean getRangesWithMetrics(
if (cardinalities.size() == 1) {
long numEntries = lowestCardinality.getKey();
double ratio = ((double) numEntries / (double) numRows);
- LOG.debug("Use of index would scan %d of %d rows, ratio %s. Threshold %2f, Using for table? %b", numEntries, numRows, ratio, threshold, ratio < threshold);
+ LOG.debug("Use of index would scan %s of %s rows, ratio %s. Threshold %2f, Using for index table? %s", numEntries, numRows, ratio, threshold, ratio < threshold);
if (ratio >= threshold) {
return false;
}
@@ -297,55 +318,59 @@ private long getNumRowsInTable(String metricsTable, Authorizations auths)
}
private List getIndexRanges(String indexTable, Multimap constraintRanges, Collection rowIDRanges, Authorizations auths)
- throws TableNotFoundException
+ throws TableNotFoundException, InterruptedException
{
- Set finalRanges = null;
- // For each column/constraint pair
+ Set finalRanges = new HashSet<>();
+ // For each column/constraint pair we submit a task to scan the index ranges
+ List>> tasks = new ArrayList<>();
+ CompletionService> executor = new ExecutorCompletionService<>(executorService);
for (Entry> constraintEntry : constraintRanges.asMap().entrySet()) {
- // Create a batch scanner against the index table, setting the ranges
- BatchScanner scanner = connector.createBatchScanner(indexTable, auths, 10);
- scanner.setRanges(constraintEntry.getValue());
-
- // Fetch the column family for this specific column
- Text family = new Text(getIndexColumnFamily(constraintEntry.getKey().getFamily().getBytes(UTF_8), constraintEntry.getKey().getQualifier().getBytes(UTF_8)).array());
- scanner.fetchColumnFamily(family);
-
- // For each entry in the scanner
- Text tmpQualifier = new Text();
- Set columnRanges = new HashSet<>();
- for (Entry entry : scanner) {
- entry.getKey().getColumnQualifier(tmpQualifier);
-
- // Add to our column ranges if it is in one of the row ID ranges
- if (inRange(tmpQualifier, rowIDRanges)) {
- columnRanges.add(new Range(tmpQualifier));
+ tasks.add(executor.submit(() -> {
+ // Create a batch scanner against the index table, setting the ranges
+ BatchScanner scan = connector.createBatchScanner(indexTable, auths, 10);
+ scan.setRanges(constraintEntry.getValue());
+
+ // Fetch the column family for this specific column
+ scan.fetchColumnFamily(new Text(Indexer.getIndexColumnFamily(constraintEntry.getKey().getFamily().getBytes(), constraintEntry.getKey().getQualifier().getBytes()).array()));
+
+ // For each entry in the scanner
+ Text tmpQualifier = new Text();
+ Set columnRanges = new HashSet<>();
+ for (Entry entry : scan) {
+ entry.getKey().getColumnQualifier(tmpQualifier);
+
+ // Add to our column ranges if it is in one of the row ID ranges
+ if (inRange(tmpQualifier, rowIDRanges)) {
+ columnRanges.add(new Range(tmpQualifier));
+ }
}
- }
-
- LOG.debug("Retrieved %d ranges for column %s", columnRanges.size(), constraintEntry.getKey().getName());
- // If finalRanges is null, we have not yet added any column ranges
- if (finalRanges == null) {
- finalRanges = new HashSet<>();
- finalRanges.addAll(columnRanges);
+ LOG.debug("Retrieved %d ranges for index column %s", columnRanges.size(), constraintEntry.getKey().getName());
+ scan.close();
+ return columnRanges;
+ }));
+ }
+ tasks.forEach(future ->
+ {
+ try {
+ // If finalRanges is null, we have not yet added any column ranges
+ if (finalRanges.isEmpty()) {
+ finalRanges.addAll(future.get());
+ }
+ else {
+ // Retain only the row IDs for this column that have already been added
+ // This is your set intersection operation!
+ finalRanges.retainAll(future.get());
+ }
}
- else {
- // Retain only the row IDs for this column that have already been added
- // This is your set intersection operation!
- finalRanges.retainAll(columnRanges);
+ catch (ExecutionException | InterruptedException e) {
+ if (e instanceof InterruptedException) {
+ Thread.currentThread().interrupt();
+ }
+ throw new PrestoException(UNEXPECTED_ACCUMULO_ERROR, "Exception when getting index ranges", e.getCause());
}
-
- // Close the scanner
- scanner.close();
- }
-
- // Return the final ranges for all constraint pairs
- if (finalRanges != null) {
- return ImmutableList.copyOf(finalRanges);
- }
- else {
- return ImmutableList.of();
- }
+ });
+ return ImmutableList.copyOf(finalRanges);
}
private static void binRanges(int numRangesPerBin, List splitRanges, List prestoSplits)
diff --git a/presto-accumulo/src/main/java/com/facebook/presto/accumulo/serializers/AccumuloRowSerializer.java b/presto-accumulo/src/main/java/com/facebook/presto/accumulo/serializers/AccumuloRowSerializer.java
index 21b060e586c8..affd8de9ba50 100644
--- a/presto-accumulo/src/main/java/com/facebook/presto/accumulo/serializers/AccumuloRowSerializer.java
+++ b/presto-accumulo/src/main/java/com/facebook/presto/accumulo/serializers/AccumuloRowSerializer.java
@@ -17,7 +17,6 @@
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.InterleavedBlockBuilder;
import com.facebook.presto.spi.type.Type;
import com.facebook.presto.spi.type.TypeUtils;
import com.facebook.presto.spi.type.VarcharType;
@@ -556,13 +555,16 @@ static Block getBlockFromMap(Type mapType, Map, ?> 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/TestAccumuloDistributedQueries.java b/presto-accumulo/src/test/java/com/facebook/presto/accumulo/TestAccumuloDistributedQueries.java
index 27f754d2f8df..f24bdc681d73 100644
--- a/presto-accumulo/src/test/java/com/facebook/presto/accumulo/TestAccumuloDistributedQueries.java
+++ b/presto-accumulo/src/test/java/com/facebook/presto/accumulo/TestAccumuloDistributedQueries.java
@@ -49,6 +49,12 @@ public void testAddColumn()
// Adding columns via SQL are not supported until adding columns with comments are supported
}
+ @Override
+ public void testDropColumn()
+ {
+ // Dropping columns are not supported by the connector
+ }
+
@Override
public void testCreateTableAsSelect()
{
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 6160553aac84..1282ca71ef43 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 3600d6b0c8f7..55db1b1e98e2 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
@@ -17,12 +17,12 @@
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.spi.type.TypeManager;
import com.facebook.presto.spi.type.TypeSignatureParameter;
import com.facebook.presto.sql.analyzer.FeaturesConfig;
-import com.facebook.presto.type.ArrayType;
import com.facebook.presto.type.TypeRegistry;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
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 e45bf6a7cd9a..596c81b0c4ea 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;
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 d4ed16987e08..dec82d6ac219 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
@@ -15,12 +15,12 @@
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.spi.type.TypeManager;
import com.facebook.presto.spi.type.TypeSignatureParameter;
import com.facebook.presto.sql.analyzer.FeaturesConfig;
-import com.facebook.presto.type.ArrayType;
import com.facebook.presto.type.TypeRegistry;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
diff --git a/presto-array/pom.xml b/presto-array/pom.xml
index 5a62826f3e01..a34e9edf331c 100644
--- a/presto-array/pom.xml
+++ b/presto-array/pom.xml
@@ -5,7 +5,7 @@
com.facebook.presto
presto-root
- 0.179-tw-0.36
+ 0.181-tw-0.37
presto-array
@@ -21,6 +21,11 @@
slice
+
+ 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 da6e0d972c36..f6bb1106315b 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
@@ -20,6 +20,7 @@ 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()
@@ -37,7 +38,7 @@ public BlockBigArray(Block block)
*/
public long sizeOf()
{
- return INSTANCE_SIZE + array.sizeOf() + sizeOfBlocks;
+ return INSTANCE_SIZE + array.sizeOf() + sizeOfBlocks + trackedObjects.sizeOf();
}
/**
@@ -60,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/ReferenceCountMap.java b/presto-array/src/main/java/com/facebook/presto/array/ReferenceCountMap.java
new file mode 100644
index 000000000000..b0c2130442ac
--- /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
+
+ 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 0685b71820cb..7e97b21ade27 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 2ca469875c64..1f82be9d18f2 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 8d15446a3e19..282236e91ab7 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 5a59fd82848e..99942fab22ca 100644
--- a/presto-benchmark/pom.xml
+++ b/presto-benchmark/pom.xml
@@ -5,7 +5,7 @@
presto-root
com.facebook.presto
- 0.179-tw-0.36
+ 0.181-tw-0.37
presto-benchmark
diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml
index 4302c3313dcc..45cac6b08720 100644
--- a/presto-benchto-benchmarks/pom.xml
+++ b/presto-benchto-benchmarks/pom.xml
@@ -4,7 +4,7 @@
com.facebook.presto
presto-root
- 0.179-tw-0.36
+ 0.181-tw-0.37
presto-benchto-benchmarks
diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml
index 26677923e7d0..d3fa947ee109 100644
--- a/presto-blackhole/pom.xml
+++ b/presto-blackhole/pom.xml
@@ -5,7 +5,7 @@
com.facebook.presto
presto-root
- 0.179-tw-0.36
+ 0.181-tw-0.37
presto-blackhole
diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml
index 6aaca8aded5a..75e1d227acf7 100644
--- a/presto-bytecode/pom.xml
+++ b/presto-bytecode/pom.xml
@@ -5,7 +5,7 @@
com.facebook.presto
presto-root
- 0.179-tw-0.36
+ 0.181-tw-0.37
presto-bytecode
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 0d0c50b015af..d56a84c43eab 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
@@ -15,6 +15,7 @@
import com.google.common.base.Joiner;
import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import org.objectweb.asm.ClassVisitor;
import org.objectweb.asm.MethodVisitor;
@@ -30,6 +31,7 @@
import static com.facebook.presto.bytecode.Access.STATIC;
import static com.facebook.presto.bytecode.Access.toAccessModifier;
import static com.facebook.presto.bytecode.ParameterizedType.type;
+import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.collect.Iterables.transform;
import static org.objectweb.asm.Opcodes.RETURN;
@@ -70,6 +72,8 @@ public MethodDefinition(
Iterable parameters
)
{
+ checkArgument(Iterables.size(parameters) <= 254, "Too many parameters for method");
+
this.declaringClass = declaringClass;
body = new BytecodeBlock();
diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml
index 7d2de9976131..2a499ac16bbb 100644
--- a/presto-cassandra/pom.xml
+++ b/presto-cassandra/pom.xml
@@ -4,7 +4,7 @@
com.facebook.presto
presto-root
- 0.179-tw-0.36
+ 0.181-tw-0.37
presto-cassandra
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 eecbbf6ae2f0..6693baff17b0 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 671fb8e8ebad..fcd4dcd5fc86 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
@@ -55,6 +55,16 @@ public RecordSink getRecordSink(ConnectorTransactionHandle transaction, Connecto
@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 362739abca76..70d415ebebba 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 f2fe188af719..c82269ea0f22 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 000000000000..57d1e55abeb0
--- /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 d68cc32448e5..3307f00f75bc 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;
@@ -47,8 +48,11 @@
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;
@@ -199,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(
cassandraSession.getTable(getTableName(handle)).getClusteringKeyColumns(),
- partitionResult.getUnenforcedConstraint());
+ partitionResult.getUnenforcedConstraint(),
+ cassandraSession.getCassandraVersion());
clusteringKeyPredicates = clusteringPredicatesExtractor.getClusteringKeyPredicates();
unenforcedConstraint = clusteringPredicatesExtractor.getUnenforcedConstraints();
}
@@ -311,4 +315,27 @@ public Optional finishCreateTable(ConnectorSession sess
{
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/CassandraSession.java b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraSession.java
index 139bd458fa3b..fa5d3d5274b3 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
@@ -19,6 +19,7 @@
import com.datastax.driver.core.ResultSet;
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;
@@ -31,6 +32,8 @@ public interface CassandraSession
{
String PRESTO_COMMENT_METADATA = "Presto Metadata:";
+ VersionNumber getCassandraVersion();
+
String getPartitioner();
Set getTokenRanges();
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 7580a59fa7ba..d0ce4dbbbb13 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
@@ -105,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();
@@ -148,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) {
@@ -163,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();
@@ -171,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 dfafda61ef6d..fe2da9a38c01 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/NativeCassandraSession.java b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/NativeCassandraSession.java
index 80c84e487b7f..bd08ff21fdbf 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
@@ -27,6 +27,7 @@
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;
@@ -62,6 +63,7 @@
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;
@@ -100,6 +102,19 @@ public NativeCassandraSession(String connectorId, JsonCodec 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()
{
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 e312b2d3feaa..26334b8434fc 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)
@@ -111,7 +115,34 @@ public static void insertIntoTableMultiPartitionClusteringKeys(CassandraSession
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)))
@@ -214,6 +245,6 @@ private static void insertTestData(CassandraSession session, SchemaTableName tab
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 1d9df2ae4f27..dd6b0f9e9f61 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
@@ -81,6 +81,7 @@ public static synchronized void start()
.withClusterName("TestCluster")
.addContactPointsWithPorts(ImmutableList.of(
new InetSocketAddress(HOST, PORT)))
+ .withMaxSchemaAgreementWaitSeconds(30)
.build();
CassandraSession session = new NativeCassandraSession(
diff --git a/presto-cassandra/src/test/java/com/facebook/presto/cassandra/TestCassandraDistributed.java b/presto-cassandra/src/test/java/com/facebook/presto/cassandra/TestCassandraDistributed.java
index 572a2018df1f..54d44a5cfa42 100644
--- a/presto-cassandra/src/test/java/com/facebook/presto/cassandra/TestCassandraDistributed.java
+++ b/presto-cassandra/src/test/java/com/facebook/presto/cassandra/TestCassandraDistributed.java
@@ -75,6 +75,12 @@ public void testRenameColumn()
// Cassandra does not support renaming columns
}
+ @Override
+ public void testDropColumn()
+ {
+ // Cassandra does not support dropping columns
+ }
+
@Override
public void testInsert()
{
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 81eb2ebc6664..7506ceffc03e 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
@@ -33,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;
@@ -201,6 +203,54 @@ 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
@@ -225,8 +275,7 @@ public void testUpperCaseNameUnescapedInCassandra()
.row("column_1", "bigint", "", "")
.build());
- // TODO replace with the Presto INSERT INTO once implemented
- session.execute("INSERT INTO keyspace_1.table_1 (column_1) VALUES (1)");
+ 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");
@@ -257,8 +306,7 @@ public void testUppercaseNameEscaped()
.row("column_2", "bigint", "", "")
.build());
- // TODO replace with the Presto INSERT INTO once implemented
- session.execute("INSERT INTO \"KEYSPACE_2\".\"TABLE_2\" (\"COLUMN_2\") VALUES (1)");
+ 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");
@@ -283,8 +331,10 @@ public void testKeyspaceNameAmbiguity()
.build(), new Duration(1, MINUTES));
// There is no way to figure out what the exactly keyspace we want to retrieve tables from
- assertQueryFails("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\\)");
+ 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\"");
@@ -311,11 +361,14 @@ public void testTableNameAmbiguity()
.build(), new Duration(1, MINUTES));
// There is no way to figure out what the exactly table is being queried
- assertQueryFails("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\\)");
- assertQueryFails("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\\)");
-
+ 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");
}
@@ -333,14 +386,123 @@ public void testColumnNameAmbiguity()
.row("table_5")
.build(), new Duration(1, MINUTES));
- assertQueryFails("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\\)");
- assertQueryFails("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\\)");
+ 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)
{
Type uuidType = createdByPresto ? createUnboundedVarcharType() : createVarcharType(36);
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 5a56a88d3d47..78e5c82610b7 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 a70dfd7fc429..59ff3b03bb86 100644
--- a/presto-cli/pom.xml
+++ b/presto-cli/pom.xml
@@ -5,7 +5,7 @@
com.facebook.presto
presto-root
- 0.179-tw-0.36
+ 0.181-tw-0.37
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 ce73ad7b7de3..79bf95a058ef 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
@@ -19,10 +19,8 @@
import com.google.common.net.HostAndPort;
import com.sun.security.auth.module.UnixSystem;
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;
@@ -34,6 +32,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;
@@ -59,7 +58,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;
@@ -116,6 +115,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);
@@ -148,22 +150,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);
@@ -173,7 +159,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);
@@ -193,15 +179,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 03b10369ebd9..ef72bdad3d7a 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;
@@ -94,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);
@@ -125,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),
@@ -136,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);
}
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 de40f0e32cc8..000000000000
--- 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/QueryRunner.java b/presto-cli/src/main/java/com/facebook/presto/cli/QueryRunner.java
index 80fa038eafab..83031ba99574 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 7d45a47cb7ae..80f65a834ec2 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 3c1bf78c27cd..e9f4275cfd64 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 cfb1b002babd..564ef54c8b3d 100644
--- a/presto-client/pom.xml
+++ b/presto-client/pom.xml
@@ -5,7 +5,7 @@
com.facebook.presto
presto-root
- 0.179-tw-0.36
+ 0.181-tw-0.37
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 000000000000..e9cc75ce70f4
--- /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 000000000000..12609e66296a
--- /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 000000000000..914def58b202
--- /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 000000000000..4b788a56535c
--- /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/SpnegoHandler.java b/presto-client/src/main/java/com/facebook/presto/client/SpnegoHandler.java
new file mode 100644
index 000000000000..cb08cb598392
--- /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 7373f17338af..785a73b9ae38 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-docs/pom.xml b/presto-docs/pom.xml
index b01a12bf2df9..3382fdd23f25 100644
--- a/presto-docs/pom.xml
+++ b/presto-docs/pom.xml
@@ -5,7 +5,7 @@
com.facebook.presto
presto-root
- 0.179-tw-0.36
+ 0.181-tw-0.37
presto-docs
@@ -44,6 +44,80 @@
+
+ org.codehaus.mojo
+ exec-maven-plugin
+
+
+ validate-reserved
+ validate
+
+ java
+
+
+ com.facebook.presto.sql.ReservedIdentifiers
+
+ validateDocs
+ ${project.basedir}/src/main/sphinx/language/reserved.rst
+
+
+
+
+ generate-thrift-idl
+ validate
+
+ java
+
+
+ com.facebook.swift.generator.swift2thrift.Main
+
+ com.facebook.presto.connector.thrift.api.PrestoThriftService
+ -recursive
+ -out
+ ${project.build.directory}/PrestoThriftService.thrift
+
+
+
+
+ validate-thrift-idl
+ validate
+
+ exec
+
+
+ diff
+
+ -b
+ -c
+ ${project.basedir}/src/main/sphinx/include/PrestoThriftService.thrift
+ ${project.build.directory}/PrestoThriftService.thrift
+
+
+
+
+
+ false
+ true
+
+
+
+ com.facebook.presto
+ presto-parser
+ ${project.version}
+
+
+ com.facebook.presto
+ presto-thrift-connector-api
+ ${project.version}
+
+
+ com.facebook.swift
+ swift2thrift-generator-cli
+ ${dep.swift.version}
+
+
+
+
io.airlift.maven.plugins
sphinx-maven-plugin
diff --git a/presto-docs/src/main/sphinx/admin/properties.rst b/presto-docs/src/main/sphinx/admin/properties.rst
index 89c4d087aa95..6284c52f07bf 100644
--- a/presto-docs/src/main/sphinx/admin/properties.rst
+++ b/presto-docs/src/main/sphinx/admin/properties.rst
@@ -366,3 +366,51 @@ Optimizer Properties
``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 f17881b387cb..a65bdc131700 100644
--- a/presto-docs/src/main/sphinx/admin/resource-groups.rst
+++ b/presto-docs/src/main/sphinx/admin/resource-groups.rst
@@ -68,6 +68,15 @@ Selector Properties
* ``source`` (optional): regex to match against source string. Defaults to ``.*``
+* ``queryType`` (optional): string to match against the type of the query submitted. The query type can be:
+ * ``DATA_DEFINITION``: Queries that alter/create/drop the metadata of schemas/tables/views, and that manage
+ prepared statements, privileges, sessions, and transactions.
+ * ``DELETE``: ``DELETE`` queries.
+ * ``DESCRIBE``: ``DESCRIBE``, ``DESCRIBE INPUT``, ``DESCRIBE OUTPUT``, and ``SHOW`` queries.
+ * ``EXPLAIN``: ``EXPLAIN`` queries.
+ * ``INSERT``: ``INSERT`` and ``CREATE TABLE AS SELECT`` queries.
+ * ``SELECT``: ``SELECT`` queries.
+
* ``group`` (required): the group these queries will run in.
Global Properties
@@ -89,7 +98,11 @@ There are three selectors that define which queries run in which resource group:
* The first selector places queries from ``bob`` into the admin group.
- * The second selector states that all queries that come from a source that includes "pipeline"
+ * The second selector states that all data definition queries that come from a source that includes "pipeline"
+ should run in the user's personal data definition group, which belongs to the
+ ``globa.data_definition`` parent group.
+
+ * The third selector states that all queries that come from a source that includes "pipeline"
should run in the user's personal pipeline group, which belongs to the ``global.pipeline``
parent group.
@@ -116,6 +129,13 @@ all other users are subject to the following limits:
"schedulingPolicy": "weighted",
"jmxExport": true,
"subGroups": [
+ {
+ "name": "data_definition_${USER}",
+ "softMemoryLimit": "10%",
+ "maxRunning": 3,
+ "maxQueued": 10,
+ "schedulingWeight": 1
+ },
{
"name": "adhoc_${USER}",
"softMemoryLimit": "10%",
@@ -157,6 +177,11 @@ all other users are subject to the following limits:
"user": "bob",
"group": "admin"
},
+ {
+ "source": ".*pipeline.*",
+ "queryType": "DATA_DEFINITION",
+ "group": "global.data_definition_${USER}"
+ },
{
"source": ".*pipeline.*",
"group": "global.pipeline.pipeline_${USER}"
diff --git a/presto-docs/src/main/sphinx/connector.rst b/presto-docs/src/main/sphinx/connector.rst
index ee10d9c29cf4..651e22ab96a0 100644
--- a/presto-docs/src/main/sphinx/connector.rst
+++ b/presto-docs/src/main/sphinx/connector.rst
@@ -25,3 +25,4 @@ from different data sources.
connector/sqlserver
connector/system
connector/tpch
+ connector/thrift
diff --git a/presto-docs/src/main/sphinx/connector/cassandra.rst b/presto-docs/src/main/sphinx/connector/cassandra.rst
index 3572e068d40c..9c93f5bbb432 100644
--- a/presto-docs/src/main/sphinx/connector/cassandra.rst
+++ b/presto-docs/src/main/sphinx/connector/cassandra.rst
@@ -230,4 +230,3 @@ Limitations
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.
-* Non-equality predicates on clustering keys are not pushed down (only ``=`` and ``IN`` are pushed down) .
diff --git a/presto-docs/src/main/sphinx/connector/hive-security.rst b/presto-docs/src/main/sphinx/connector/hive-security.rst
index cb09f5a7add4..26b1f1edf069 100644
--- a/presto-docs/src/main/sphinx/connector/hive-security.rst
+++ b/presto-docs/src/main/sphinx/connector/hive-security.rst
@@ -19,7 +19,8 @@ Property Value Description
================================================== ============================================================
``legacy`` (default value) Few authorization checks are enforced, thus allowing most
operations. The config properties ``hive.allow-drop-table``,
- ``hive.allow-rename-table``, ``hive.allow-add-column`` and
+ ``hive.allow-rename-table``, ``hive.allow-add-column``,
+ ``hive.allow-drop-column`` and
``hive.allow-rename-column`` are used.
``read-only`` Operations that read data or metadata, such as ``SELECT``,
diff --git a/presto-docs/src/main/sphinx/connector/thrift.rst b/presto-docs/src/main/sphinx/connector/thrift.rst
new file mode 100644
index 000000000000..715f2bec2fec
--- /dev/null
+++ b/presto-docs/src/main/sphinx/connector/thrift.rst
@@ -0,0 +1,96 @@
+================
+Thrift Connector
+================
+
+The Thrift connector makes it possible to integrate with external storage systems
+without a custom Presto connector implementation.
+
+In order to use the Thrift connector with an external system, you need to implement
+the ``PrestoThriftService`` interface, found below. Next, you configure the Thrift connector
+to point to a set of machines, called Thrift servers, that implement the interface.
+As part of the interface implementation, the Thrift servers will provide metadata,
+splits and data. The Thrift server instances are assumed to be stateless and independent
+from each other.
+
+Configuration
+-------------
+
+To configure the Thrift connector, create a catalog properties file
+``etc/catalog/thrift.properties`` with the following content,
+replacing the properties as appropriate:
+
+.. code-block:: none
+
+ connector.name=presto-thrift
+ static-location.hosts=host:port,host:port
+
+Multiple Thrift Systems
+^^^^^^^^^^^^^^^^^^^^^^^
+
+You can have as many catalogs as you need, so if you have additional
+Thrift systems to connect to, simply add another properties file to ``etc/catalog``
+with a different name (making sure it ends in ``.properties``).
+
+Configuration Properties
+------------------------
+
+The following configuration properties are available:
+
+=========================================== ==============================================================
+Property Name Description
+=========================================== ==============================================================
+``static-location.hosts`` Location of Thrift servers
+``presto-thrift.max-response-size`` Maximum size of a response from thrift server
+``presto-thrift.metadata-refresh-threads`` Number of refresh threads for metadata cache
+=========================================== ==============================================================
+
+``static-location.hosts``
+^^^^^^^^^^^^^^^^^^^^^^^^^
+
+Comma-separated list of thrift servers in the form of ``host:port``. For example:
+
+.. code-block:: none
+
+ static-location.hosts=192.0.2.3:7777,192.0.2.4:7779
+
+This property is required; there is no default.
+
+``presto-thrift.max-response-size``
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+Maximum size of a data response that the connector accepts. This value is sent
+by the connector to the Thrift server when requesting data, allowing it to size
+the response appropriately.
+
+This property is optional; the default is ``16MB``.
+
+``presto-thrift.metadata-refresh-threads``
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+Number of refresh threads for metadata cache.
+
+This property is optional; the default is ``1``.
+
+Thrift Client Properties
+^^^^^^^^^^^^^^^^^^^^^^^^
+
+The following properties allow configuring the Thrift client used by the connector:
+
+===================================================== =================== =============
+Property Name Description Default Value
+===================================================== =================== =============
+``PrestoThriftService.thrift.client.connect-timeout`` Connect timeout ``500ms``
+``PrestoThriftService.thrift.client.max-frame-size`` Max frame size ``16777216``
+``PrestoThriftService.thrift.client.read-timeout`` Read timeout ``10s``
+``PrestoThriftService.thrift.client.receive-timeout`` Receive timeout ``1m``
+``PrestoThriftService.thrift.client.socks-proxy`` Socks proxy address ``null``
+``PrestoThriftService.thrift.client.write-timeout`` Write timeout ``1m``
+===================================================== =================== =============
+
+Thrift IDL File
+---------------
+
+The following IDL describes the ``PrestoThriftService`` that must be implemented:
+
+.. literalinclude:: /include/PrestoThriftService.thrift
+ :language: thrift
diff --git a/presto-docs/src/main/sphinx/functions/binary.rst b/presto-docs/src/main/sphinx/functions/binary.rst
index e0cb5897f514..a822a3ca0f70 100644
--- a/presto-docs/src/main/sphinx/functions/binary.rst
+++ b/presto-docs/src/main/sphinx/functions/binary.rst
@@ -1,6 +1,11 @@
-================
-Binary Functions
-================
+==============================
+Binary Functions and Operators
+==============================
+
+Binary Operators
+----------------
+
+The ``||`` operator performs concatenation.
Binary Functions
----------------
@@ -10,6 +15,13 @@ Binary Functions
Returns the length of ``binary`` in bytes.
+.. function:: concat(binary1, ..., binaryN) -> varbinary
+ :noindex:
+
+ Returns the concatenation of ``binary1``, ``binary2``, ``...``, ``binaryN``.
+ This function provides the same functionality as the
+ SQL-standard concatenation operator (``||``).
+
.. function:: to_base64(binary) -> varchar
Encodes ``binary`` into a base64 string representation.
diff --git a/presto-docs/src/main/sphinx/functions/datetime.rst b/presto-docs/src/main/sphinx/functions/datetime.rst
index defac4f571cf..728f5a26a329 100644
--- a/presto-docs/src/main/sphinx/functions/datetime.rst
+++ b/presto-docs/src/main/sphinx/functions/datetime.rst
@@ -53,6 +53,10 @@ Date and Time Functions
Returns the current time zone in the format defined by IANA
(e.g., ``America/Los_Angeles``) or as fixed offset from UTC (e.g., ``+08:35``)
+.. function:: date(x) -> date
+
+ This is an alias for ``CAST(x AS date)``.
+
.. function:: from_iso8601_timestamp(string) -> timestamp with time zone
Parses the ISO 8601 formatted ``string`` into a ``timestamp with time zone``.
diff --git a/presto-docs/src/main/sphinx/include/PrestoThriftService.thrift b/presto-docs/src/main/sphinx/include/PrestoThriftService.thrift
new file mode 100644
index 000000000000..9d2052d6c92b
--- /dev/null
+++ b/presto-docs/src/main/sphinx/include/PrestoThriftService.thrift
@@ -0,0 +1,325 @@
+namespace java.swift com.facebook.presto.connector.thrift.api
+
+
+enum PrestoThriftBound {
+ BELOW=1, EXACTLY=2, ABOVE=3
+}
+
+exception PrestoThriftServiceException {
+ 1: string message;
+ 2: bool retryable;
+}
+
+struct PrestoThriftNullableSchemaName {
+ 1: optional string schemaName;
+}
+
+struct PrestoThriftSchemaTableName {
+ 1: string schemaName;
+ 2: string tableName;
+}
+
+struct PrestoThriftColumnMetadata {
+ 1: string name;
+ 2: string type;
+ 3: optional string comment;
+ 4: bool hidden;
+}
+
+struct PrestoThriftNullableColumnSet {
+ 1: optional set columns;
+}
+
+
+/**
+ * Set that either includes all values, or excludes all values.
+ */
+struct PrestoThriftAllOrNoneValueSet {
+ 1: bool all;
+}
+
+
+/**
+ * Elements of {@code nulls} array determine if a value for a corresponding row is null.
+ * Elements of {@code ints} array are values for each row. If row is null then value is ignored.
+ */
+struct PrestoThriftInteger {
+ 1: optional list nulls;
+ 2: optional list ints;
+}
+
+
+/**
+ * Elements of {@code nulls} array determine if a value for a corresponding row is null.
+ * Elements of {@code longs} array are values for each row. If row is null then value is ignored.
+ */
+struct PrestoThriftBigint {
+ 1: optional list nulls;
+ 2: optional list longs;
+}
+
+
+/**
+ * Elements of {@code nulls} array determine if a value for a corresponding row is null.
+ * Elements of {@code doubles} array are values for each row. If row is null then value is ignored.
+ */
+struct PrestoThriftDouble {
+ 1: optional list nulls;
+ 2: optional list doubles;
+}
+
+
+/**
+ * Elements of {@code nulls} array determine if a value for a corresponding row is null.
+ * Each elements of {@code sizes} array contains the length in bytes for the corresponding element.
+ * If row is null then the corresponding element in {@code sizes} is ignored.
+ * {@code bytes} array contains uft8 encoded byte values.
+ * Values for all rows are written to {@code bytes} array one after another.
+ * The total number of bytes must be equal to the sum of all sizes.
+ */
+struct PrestoThriftVarchar {
+ 1: optional list nulls;
+ 2: optional list sizes;
+ 3: optional binary bytes;
+}
+
+
+/**
+ * Elements of {@code nulls} array determine if a value for a corresponding row is null.
+ * Elements of {@code booleans} array are values for each row. If row is null then value is ignored.
+ */
+struct PrestoThriftBoolean {
+ 1: optional list nulls;
+ 2: optional list booleans;
+}
+
+
+/**
+ * Elements of {@code nulls} array determine if a value for a corresponding row is null.
+ * Elements of {@code dates} array are date values for each row represented as the number
+ * of days passed since 1970-01-01.
+ * If row is null then value is ignored.
+ */
+struct PrestoThriftDate {
+ 1: optional list nulls;
+ 2: optional list dates;
+}
+
+
+/**
+ * Elements of {@code nulls} array determine if a value for a corresponding row is null.
+ * Elements of {@code timestamps} array are values for each row represented as the number
+ * of milliseconds passed since 1970-01-01T00:00:00 UTC.
+ * If row is null then value is ignored.
+ */
+struct PrestoThriftTimestamp {
+ 1: optional list nulls;
+ 2: optional list timestamps;
+}
+
+
+/**
+ * Elements of {@code nulls} array determine if a value for a corresponding row is null.
+ * Each elements of {@code sizes} array contains the length in bytes for the corresponding element.
+ * If row is null then the corresponding element in {@code sizes} is ignored.
+ * {@code bytes} array contains uft8 encoded byte values for string representation of json.
+ * Values for all rows are written to {@code bytes} array one after another.
+ * The total number of bytes must be equal to the sum of all sizes.
+ */
+struct PrestoThriftJson {
+ 1: optional list nulls;
+ 2: optional list sizes;
+ 3: optional binary bytes;
+}
+
+
+/**
+ * Elements of {@code nulls} array determine if a value for a corresponding row is null.
+ * Each elements of {@code sizes} array contains the length in bytes for the corresponding element.
+ * If row is null then the corresponding element in {@code sizes} is ignored.
+ * {@code bytes} array contains encoded byte values for HyperLogLog representation as defined in
+ * Airlift specification: href="https://github.com/airlift/airlift/blob/master/stats/docs/hll.md
+ * Values for all rows are written to {@code bytes} array one after another.
+ * The total number of bytes must be equal to the sum of all sizes.
+ */
+struct PrestoThriftHyperLogLog {
+ 1: optional list nulls;
+ 2: optional list sizes;
+ 3: optional binary bytes;
+}
+
+
+/**
+ * Elements of {@code nulls} array determine if a value for a corresponding row is null.
+ * Each elements of {@code sizes} array contains the number of elements in the corresponding values array.
+ * If row is null then the corresponding element in {@code sizes} is ignored.
+ * {@code values} is a bigint block containing array elements one after another for all rows.
+ * The total number of elements in bigint block must be equal to the sum of all sizes.
+ */
+struct PrestoThriftBigintArray {
+ 1: optional list nulls;
+ 2: optional list sizes;
+ 3: optional PrestoThriftBigint values;
+}
+
+struct PrestoThriftId {
+ 1: binary id;
+}
+
+struct PrestoThriftHostAddress {
+ 1: string host;
+ 2: i32 port;
+}
+
+struct PrestoThriftTableMetadata {
+ 1: PrestoThriftSchemaTableName schemaTableName;
+ 2: list columns;
+ 3: optional string comment;
+}
+
+struct PrestoThriftBlock {
+ 1: optional PrestoThriftInteger integerData;
+ 2: optional PrestoThriftBigint bigintData;
+ 3: optional PrestoThriftDouble doubleData;
+ 4: optional PrestoThriftVarchar varcharData;
+ 5: optional PrestoThriftBoolean booleanData;
+ 6: optional PrestoThriftDate dateData;
+ 7: optional PrestoThriftTimestamp timestampData;
+ 8: optional PrestoThriftJson jsonData;
+ 9: optional PrestoThriftHyperLogLog hyperLogLogData;
+ 10: optional PrestoThriftBigintArray bigintArrayData;
+}
+
+
+/**
+ * LOWER UNBOUNDED is specified with an empty value and an ABOVE bound
+ * UPPER UNBOUNDED is specified with an empty value and a BELOW bound
+ */
+struct PrestoThriftMarker {
+ 1: optional PrestoThriftBlock value;
+ 2: PrestoThriftBound bound;
+}
+
+struct PrestoThriftNullableToken {
+ 1: optional PrestoThriftId token;
+}
+
+struct PrestoThriftSplit {
+ 1: PrestoThriftId splitId;
+ 2: list hosts;
+}
+
+struct PrestoThriftPageResult {
+ /**
+ * Returns data in a columnar format.
+ * Columns in this list must be in the order they were requested by the engine.
+ */
+ 1: list columnBlocks;
+ 2: i32 rowCount;
+ 3: optional PrestoThriftId nextToken;
+}
+
+struct PrestoThriftNullableTableMetadata {
+ 1: optional PrestoThriftTableMetadata tableMetadata;
+}
+
+
+/**
+ * A set containing values that are uniquely identifiable.
+ * Assumes an infinite number of possible values. The values may be collectively included (aka whitelist)
+ * or collectively excluded (aka !whitelist).
+ * This structure is used with comparable, but not orderable types like "json", "map".
+ */
+struct PrestoThriftEquatableValueSet {
+ 1: bool whiteList;
+ 2: list values;
+}
+
+struct PrestoThriftRange {
+ 1: PrestoThriftMarker low;
+ 2: PrestoThriftMarker high;
+}
+
+struct PrestoThriftSplitBatch {
+ 1: list splits;
+ 2: optional PrestoThriftId nextToken;
+}
+
+
+/**
+ * A set containing zero or more Ranges of the same type over a continuous space of possible values.
+ * Ranges are coalesced into the most compact representation of non-overlapping Ranges.
+ * This structure is used with comparable and orderable types like bigint, integer, double, varchar, etc.
+ */
+struct PrestoThriftRangeValueSet {
+ 1: list ranges;
+}
+
+struct PrestoThriftValueSet {
+ 1: optional PrestoThriftAllOrNoneValueSet allOrNoneValueSet;
+ 2: optional PrestoThriftEquatableValueSet equatableValueSet;
+ 3: optional PrestoThriftRangeValueSet rangeValueSet;
+}
+
+struct PrestoThriftDomain {
+ 1: PrestoThriftValueSet valueSet;
+ 2: bool nullAllowed;
+}
+
+struct PrestoThriftTupleDomain {
+ /**
+ * Return a map of column names to constraints.
+ */
+ 1: optional map domains;
+}
+
+/**
+ * Presto Thrift service definition.
+ * This thrift service needs to be implemented in order to be used with Thrift Connector.
+ */
+service PrestoThriftService {
+ /**
+ * Returns available schema names.
+ */
+ list prestoListSchemaNames() throws (1: PrestoThriftServiceException ex1);
+
+ /**
+ * Returns tables for the given schema name.
+ *
+ * @param schemaNameOrNull a structure containing schema name or {@literal null}
+ * @return a list of table names with corresponding schemas. If schema name is null then returns
+ * a list of tables for all schemas. Returns an empty list if a schema does not exist
+ */
+ list prestoListTables(1: PrestoThriftNullableSchemaName schemaNameOrNull) throws (1: PrestoThriftServiceException ex1);
+
+ /**
+ * Returns metadata for a given table.
+ *
+ * @param schemaTableName schema and table name
+ * @return metadata for a given table, or a {@literal null} value inside if it does not exist
+ */
+ PrestoThriftNullableTableMetadata prestoGetTableMetadata(1: PrestoThriftSchemaTableName schemaTableName) throws (1: PrestoThriftServiceException ex1);
+
+ /**
+ * Returns a batch of splits.
+ *
+ * @param schemaTableName schema and table name
+ * @param desiredColumns a superset of columns to return; empty set means "no columns", {@literal null} set means "all columns"
+ * @param outputConstraint constraint on the returned data
+ * @param maxSplitCount maximum number of splits to return
+ * @param nextToken token from a previous split batch or {@literal null} if it is the first call
+ * @return a batch of splits
+ */
+ PrestoThriftSplitBatch prestoGetSplits(1: PrestoThriftSchemaTableName schemaTableName, 2: PrestoThriftNullableColumnSet desiredColumns, 3: PrestoThriftTupleDomain outputConstraint, 4: i32 maxSplitCount, 5: PrestoThriftNullableToken nextToken) throws (1: PrestoThriftServiceException ex1);
+
+ /**
+ * Returns a batch of rows for the given split.
+ *
+ * @param splitId split id as returned in split batch
+ * @param columns a list of column names to return
+ * @param maxBytes maximum size of returned data in bytes
+ * @param nextToken token from a previous batch or {@literal null} if it is the first call
+ * @return a batch of table data
+ */
+ PrestoThriftPageResult prestoGetRows(1: PrestoThriftId splitId, 2: list columns, 3: i64 maxBytes, 4: PrestoThriftNullableToken nextToken) throws (1: PrestoThriftServiceException ex1);
+}
diff --git a/presto-docs/src/main/sphinx/installation/jdbc.rst b/presto-docs/src/main/sphinx/installation/jdbc.rst
index b02a98d3028a..c31202191c14 100644
--- a/presto-docs/src/main/sphinx/installation/jdbc.rst
+++ b/presto-docs/src/main/sphinx/installation/jdbc.rst
@@ -4,6 +4,20 @@ JDBC Driver
Presto can be accessed from Java using the JDBC driver.
Download :maven_download:`jdbc` and add it to the class path of your Java application.
+
+The driver is also available from Maven Central:
+
+.. parsed-literal::
+
+
+ com.facebook.presto
+ presto-jdbc
+ \ |version|\
+
+
+Connecting
+----------
+
The following JDBC URL formats are supported:
.. code-block:: none
@@ -19,3 +33,61 @@ and the schema ``sales``:
.. code-block:: none
jdbc:presto://example.net:8080/hive/sales
+
+The above URL can be used as follows to create a connection:
+
+.. code-block:: java
+
+ String url = "jdbc:presto://example.net:8080/hive/sales";
+ Connection connection = DriverManager.getConnection(url, "test", null);
+
+Connection Parameters
+---------------------
+
+The driver supports various parameters that may be set as URL parameters
+or as properties passed to ``DriverManager``. Both of the following
+examples are equivalent:
+
+.. code-block:: java
+
+ // URL parameters
+ String url = "jdbc:presto://example.net:8080/hive/sales";
+ Properties properties = new Properties();
+ properties.setProperty("user", "test");
+ properties.setProperty("password", "secret");
+ properties.setProperty("SSL", "true");
+ Connection connection = DriverManager.getConnection(url, properties);
+
+ // properties
+ String url = "jdbc:presto://example.net:8080/hive/sales?user=test&password=secret&SSL=true";
+ Connection connection = DriverManager.getConnection(url);
+
+These methods may be mixed; some parameters may be specified in the URL
+while others are specified using properties. However, the same parameter
+may not be specified using both methods.
+
+Parameter Reference
+-------------------
+
+================================= =======================================================================
+Name Description
+================================= =======================================================================
+``user`` Username to use for authentication and authorization.
+``password`` Password to use for LDAP authentication.
+``socksProxy`` SOCKS proxy host and port. Example: ``localhost:1080``
+``httpProxy`` HTTP proxy host and port. Example: ``localhost:8888``
+``SSL`` Use HTTPS for connections
+``SSLTrustStorePath`` The location of the Java TrustStore file that will be used
+ to validate HTTPS server certificates.
+``SSLTrustStorePassword`` The password for the TrustStore.
+``KerberosRemoteServiceName`` Presto coordinator Kerberos service name. This parameter is
+ required for Kerberos authentiation.
+``KerberosPrincipal`` The principal to use when authenticating to the Presto coordinator.
+``KerberosUseCanonicalHostname`` Use the canonical hostname of the Presto coordinator for the Kerberos
+ service principal by first resolving the hostname to an IP address
+ and then doing a reverse DNS lookup for that IP address.
+ This is enabled by default.
+``KerberosConfigPath`` Kerberos configuration file.
+``KerberosKeytabPath`` Kerberos keytab file.
+``KerberosCredentialCachePath`` Kerberos credential cache.
+================================= =======================================================================
diff --git a/presto-docs/src/main/sphinx/language.rst b/presto-docs/src/main/sphinx/language.rst
index 63f9d14fdeaa..a19e4b60733b 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 000000000000..9cf05af04d9e
--- /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/release.rst b/presto-docs/src/main/sphinx/release.rst
index 2bde0e3eeab3..a6f2bdfb3b37 100644
--- a/presto-docs/src/main/sphinx/release.rst
+++ b/presto-docs/src/main/sphinx/release.rst
@@ -5,6 +5,8 @@ Release Notes
.. toctree::
:maxdepth: 1
+ release/release-0.181
+ release/release-0.180
release/release-0.179
release/release-0.178
release/release-0.177
diff --git a/presto-docs/src/main/sphinx/release/release-0.180.rst b/presto-docs/src/main/sphinx/release/release-0.180.rst
new file mode 100644
index 000000000000..4befdab62818
--- /dev/null
+++ b/presto-docs/src/main/sphinx/release/release-0.180.rst
@@ -0,0 +1,63 @@
+=============
+Release 0.180
+=============
+
+General Changes
+---------------
+
+* Fix a rare bug where rows containing only ``null`` values are not returned
+ to the client. This only occurs when an entire result page contains only
+ ``null`` values. The only known case is a query over an ORC encoded Hive table
+ that does not perform any transformation of the data.
+* Fix incorrect results when performing comparisons between values of approximate
+ data types (``REAL``, ``DOUBLE``) and columns of certain exact numeric types
+ (``INTEGER``, ``BIGINT``, ``DECIMAL``).
+* Fix memory accounting for :func:`min_by` and :func:`max_by` on complex types.
+* Fix query failure due to ``NoClassDefFoundError`` when scalar functions declared
+ in plugins are implemented with instance methods.
+* Improve performance of map subscript from O(n) to O(1) in all cases. Previously, only maps
+ produced by certain functions and readers could take advantage of this improvement.
+* Skip unknown costs in ``EXPLAIN`` output.
+* Support :doc:`/security/internal-communication` between Presto nodes.
+* Add initial support for ``CROSS JOIN`` against ``LATERAL`` derived tables.
+* Add support for ``VARBINARY`` concatenation.
+* Add :doc:`/connector/thrift` that makes it possible to use Presto with
+ external systems without the need to implement a custom connector.
+* Add experimental ``/v1/resourceGroupState`` REST endpoint on coordinator.
+
+Hive Changes
+------------
+
+* Fix skipping short decimal values in the optimized Parquet reader
+ when they are backed by the ``int32`` or ``int64`` types.
+* Ignore partition bucketing if table is not bucketed. This allows dropping
+ the bucketing from table metadata but leaving it for old partitions.
+* Improve error message for Hive partitions dropped during execution.
+* The optimized RCFile writer is enabled by default, but can be disabled
+ with the ``hive.rcfile-optimized-writer.enabled`` config option.
+ The writer supports validation which reads back the entire file after
+ writing. Validation is disabled by default, but can be enabled with the
+ ``hive.rcfile.writer.validate`` config option.
+
+Cassandra Changes
+-----------------
+
+* Add support for ``INSERT``.
+* Add support for pushdown of non-equality predicates on clustering keys.
+
+JDBC Driver Changes
+-------------------
+
+* Add support for authenticating using Kerberos.
+* Allow configuring SSL/TLS and Kerberos properties on a per-connection basis.
+* Add support for executing queries using a SOCKS or HTTP proxy.
+
+CLI Changes
+-----------
+
+* Add support for executing queries using an HTTP proxy.
+
+SPI Changes
+-----------
+
+* Add running time limit and queued time limit to ``ResourceGroupInfo``.
diff --git a/presto-docs/src/main/sphinx/release/release-0.181.rst b/presto-docs/src/main/sphinx/release/release-0.181.rst
new file mode 100644
index 000000000000..5e473f888232
--- /dev/null
+++ b/presto-docs/src/main/sphinx/release/release-0.181.rst
@@ -0,0 +1,66 @@
+=============
+Release 0.181
+=============
+
+General Changes
+---------------
+
+* Fix query failure and memory usage tracking when query contains
+ :func:`transform_keys` or :func:`transform_values`.
+* Prevent ``CREATE TABLE IF NOT EXISTS`` queries from ever failing with *"Table already exists"*.
+* Fix query failure when ``ORDER BY`` expressions reference columns that are used in
+ the ``GROUP BY`` clause by their fully-qualified name.
+* Fix excessive GC overhead caused by large arrays and maps containing ``VARCHAR`` elements.
+* Improve error handling when passing too many arguments to various
+ functions or operators that take a variable number of arguments.
+* Improve performance of ``count(*)`` aggregations over subqueries with known
+ constant cardinality.
+* Add ``VERBOSE`` option for :doc:`/sql/explain-analyze` that provides additional
+ low-level details about query performance.
+* Add per-task distribution information to the output of ``EXPLAIN ANALYZE``.
+* Add support for ``DROP COLUMN`` in :doc:`/sql/alter-table`.
+* Change local scheduler to prevent starvation of long running queries
+ when the cluster is under constant load from short queries. The new
+ behavior is disabled by default and can be enabled by setting the
+ config property ``task.level-absolute-priority=true``.
+* Improve the fairness of the local scheduler such that long-running queries
+ which spend more time on the CPU per scheduling quanta (e.g., due to
+ slow connectors) do not get a disproportionate share of CPU. The new
+ behavior is disabled by default and can be enabled by setting the
+ config property ``task.legacy-scheduling-behavior=false``.
+* Add a config option to control the prioritization of queries based on
+ elapsed scheduled time. The ``task.level-time-multiplier`` property
+ controls the target scheduled time of a level relative to the next
+ level. Higher values for this property increase the fraction of CPU
+ that will be allocated to shorter queries. This config property only
+ has an effect when ``task.level-absolute-priority=true`` and
+ ``task.legacy-scheduling-behavior=false``.
+
+Hive Changes
+------------
+
+* Fix potential native memory leak when writing tables using RCFile.
+* Correctly categorize certain errors when writing tables using RCFile.
+* Decrease the number of file system metadata calls when reading tables.
+* Add support for dropping columns.
+
+JDBC Driver Changes
+-------------------
+
+* Add support for query cancellation using ``Statement.cancel()``.
+
+PostgreSQL Changes
+------------------
+
+* Add support for operations on external tables.
+
+Accumulo Changes
+----------------
+
+* Improve query performance by scanning index ranges in parallel.
+
+SPI Changes
+-----------
+
+* Fix regression that broke serialization for ``SchemaTableName``.
+* Add access control check for ``DROP COLUMN``.
diff --git a/presto-docs/src/main/sphinx/security.rst b/presto-docs/src/main/sphinx/security.rst
index f75e13ab52bb..27b08492d4d0 100644
--- a/presto-docs/src/main/sphinx/security.rst
+++ b/presto-docs/src/main/sphinx/security.rst
@@ -10,3 +10,4 @@ Security
security/ldap
security/tls
security/built-in-system-access-control
+ security/internal-communication
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 000000000000..e3186a45cc42
--- /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/alter-table.rst b/presto-docs/src/main/sphinx/sql/alter-table.rst
index 16c976884096..4564863e0963 100644
--- a/presto-docs/src/main/sphinx/sql/alter-table.rst
+++ b/presto-docs/src/main/sphinx/sql/alter-table.rst
@@ -9,6 +9,7 @@ Synopsis
ALTER TABLE name RENAME TO new_name
ALTER TABLE name ADD COLUMN column_name data_type
+ ALTER TABLE name DROP COLUMN column_name
ALTER TABLE name RENAME COLUMN column_name TO new_column_name
Description
@@ -27,6 +28,10 @@ Add column ``zip`` to the ``users`` table::
ALTER TABLE users ADD COLUMN zip varchar;
+Drop column ``zip`` from the ``users`` table::
+
+ ALTER TABLE users DROP COLUMN zip;
+
Rename column ``id`` to ``user_id`` in the ``users`` table::
ALTER TABLE users RENAME COLUMN id TO user_id;
diff --git a/presto-docs/src/main/sphinx/sql/explain-analyze.rst b/presto-docs/src/main/sphinx/sql/explain-analyze.rst
index 910811bc77eb..8ae2fa9171cb 100644
--- a/presto-docs/src/main/sphinx/sql/explain-analyze.rst
+++ b/presto-docs/src/main/sphinx/sql/explain-analyze.rst
@@ -7,7 +7,7 @@ Synopsis
.. code-block:: none
- EXPLAIN ANALYZE statement
+ EXPLAIN ANALYZE [VERBOSE] statement
Description
-----------
@@ -15,6 +15,9 @@ Description
Execute the statement and show the distributed execution plan of the statement
along with the cost of each operation.
+The ``VERBOSE`` option will give more detailed information and low-level statistics;
+understanding these may require knowledge of Presto internals and implementation details.
+
.. note::
The stats may not be entirely accurate, especially for queries that complete quickly.
@@ -69,6 +72,29 @@ relevant plan nodes). Such statistics are useful when one wants to detect data a
orderdate := tpch:orderdate
clerk := tpch:clerk
+When the ``VERBOSE`` option is used, some operators may report additional information.
+For example, the window function operator will output the following:
+
+.. code-block:: none
+
+ EXPLAIN ANALYZE VERBOSE SELECT count(clerk) OVER() FROM orders WHERE orderdate > date '1995-01-01';
+
+ Query Plan
+ -----------------------------------------------------------------------------------------------
+ ...
+ - Window[] => [clerk:varchar(15), count:bigint]
+ Cost: {rows: ?, bytes: ?}
+ CPU fraction: 75.93%, Output: 8130 rows (230.24kB)
+ Input avg.: 8130.00 lines, Input std.dev.: 0.00%
+ Active Drivers: [ 1 / 1 ]
+ Index size: std.dev.: 0.00 bytes , 0.00 rows
+ Index count per driver: std.dev.: 0.00
+ Rows per driver: std.dev.: 0.00
+ Size of partition: std.dev.: 0.00
+ count := count("clerk")
+ ...
+
+
See Also
--------
diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml
index 449aaf26b8a2..601f42469d35 100644
--- a/presto-example-http/pom.xml
+++ b/presto-example-http/pom.xml
@@ -4,7 +4,7 @@
com.facebook.presto
presto-root
- 0.179-tw-0.36
+ 0.181-tw-0.37
presto-example-http
diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml
index 3356b2574997..f509c2b7401d 100644
--- a/presto-hive-hadoop2/pom.xml
+++ b/presto-hive-hadoop2/pom.xml
@@ -5,7 +5,7 @@
com.facebook.presto
presto-root
- 0.179-tw-0.36
+ 0.181-tw-0.37
presto-hive-hadoop2
diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml
index 137afd1ee848..4c2783422b4a 100644
--- a/presto-hive/pom.xml
+++ b/presto-hive/pom.xml
@@ -5,7 +5,7 @@
com.facebook.presto
presto-root
- 0.179-tw-0.36
+ 0.181-tw-0.37
presto-hive
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 10f65fc22e6a..43f6536e28ac 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
@@ -85,6 +85,7 @@
import static com.facebook.presto.hive.HiveUtil.isLzopIndexFile;
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;
@@ -296,6 +297,7 @@ private CompletableFuture> loadSplits()
file.getBlockLocations(),
0,
file.getLen(),
+ file.getLen(),
files.getSchema(),
files.getPartitionKeys(),
splittable,
@@ -338,7 +340,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);
@@ -354,7 +356,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);
@@ -380,6 +382,7 @@ private void loadPartition(HivePartitionMetadata partition)
file.getBlockLocations(),
0,
file.getLen(),
+ file.getLen(),
iterator.getSchema(),
iterator.getPartitionKeys(),
splittable,
@@ -410,6 +413,7 @@ private void loadPartition(HivePartitionMetadata partition)
file.getBlockLocations(),
0,
file.getLen(),
+ file.getLen(),
iterator.getSchema(),
iterator.getPartitionKeys(),
splittable,
@@ -445,6 +449,7 @@ private boolean addSplitsToSource(
targetFilesystem.getFileBlockLocations(file, split.getStart(), split.getLength()),
split.getStart(),
split.getLength(),
+ file.getLen(),
schema,
partitionKeys,
false,
@@ -531,6 +536,7 @@ private Iterator createHiveSplitIterator(
BlockLocation[] blockLocations,
long start,
long length,
+ long fileSize,
Properties schema,
List partitionKeys,
boolean splittable,
@@ -604,6 +610,7 @@ protected HiveSplit computeNext()
path,
blockLocation.getOffset() + chunkOffset,
chunkLength,
+ fileSize,
schema,
partitionKeys,
addresses,
@@ -645,6 +652,7 @@ protected HiveSplit computeNext()
path,
start,
length,
+ fileSize,
schema,
partitionKeys,
addresses,
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
index 29cf25971123..1184b19464ff 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/FileFormatDataSourceStats.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/FileFormatDataSourceStats.java
@@ -25,6 +25,7 @@ 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);
@@ -44,6 +45,13 @@ public DistributionStat getLoadedBlockBytes()
return loadedBlockBytes;
}
+ @Managed
+ @Nested
+ public DistributionStat getMaxCombinedBytesPerRow()
+ {
+ return maxCombinedBytesPerRow;
+ }
+
@Managed
@Nested
public TimeStat get0Bto100KB()
@@ -93,4 +101,9 @@ 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/GenericHiveRecordCursorProvider.java b/presto-hive/src/main/java/com/facebook/presto/hive/GenericHiveRecordCursorProvider.java
index 08aeb7575bb1..ae5eaa8d8f7a 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/GenericHiveRecordCursorProvider.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/GenericHiveRecordCursorProvider.java
@@ -53,6 +53,7 @@ public Optional createRecordCursor(
Path path,
long start,
long length,
+ long fileSize,
Properties schema,
List columns,
TupleDomain effectivePredicate,
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 69f500f77dac..6b277f414129 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);
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 5b15b57a8e5f..2a5c7713b4c7 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
@@ -98,8 +98,10 @@ 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 rcfileOptimizedWriterEnabled;
+ private boolean rcfileOptimizedWriterEnabled = true;
+ private boolean rcfileWriterValidate;
private HiveMetastoreAuthenticationType hiveMetastoreAuthenticationType = HiveMetastoreAuthenticationType.NONE;
private HdfsAuthenticationType hdfsAuthenticationType = HdfsAuthenticationType.NONE;
@@ -664,6 +666,19 @@ public HiveClientConfig setOrcStreamBufferSize(DataSize orcStreamBufferSize)
return this;
}
+ @NotNull
+ public DataSize getOrcMaxReadBlockSize()
+ {
+ return orcMaxReadBlockSize;
+ }
+
+ @Config("hive.orc.max-read-block-size")
+ public HiveClientConfig setOrcMaxReadBlockSize(DataSize orcMaxReadBlockSize)
+ {
+ this.orcMaxReadBlockSize = orcMaxReadBlockSize;
+ return this;
+ }
+
public boolean isOrcBloomFiltersEnabled()
{
return orcBloomFiltersEnabled;
@@ -690,6 +705,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;
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 46be3f04f0f4..1d1e6ad853ed 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
@@ -85,7 +85,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(PooledHiveMetastoreClientFactory.class).in(Scopes.SINGLETON);
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveErrorCode.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveErrorCode.java
index 2dd754b01f86..b46090523561 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveErrorCode.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveErrorCode.java
@@ -54,7 +54,8 @@ public enum HiveErrorCode
HIVE_WRITER_DATA_ERROR(27, EXTERNAL),
HIVE_INVALID_BUCKET_FILES(28, EXTERNAL),
HIVE_EXCEEDED_PARTITION_LIMIT(29, USER_ERROR),
- HIVE_WRITE_VALIDATION_FAILED(30, INTERNAL_ERROR);
+ HIVE_WRITE_VALIDATION_FAILED(30, INTERNAL_ERROR),
+ HIVE_PARTITION_DROPPED_DURING_QUERY(31, EXTERNAL);
private final ErrorCode errorCode;
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 e48b429c6c85..23563bf47d0c 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 12639da0159c..40ec01e774ca 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
@@ -133,6 +133,7 @@
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;
@@ -580,6 +581,15 @@ public void renameColumn(ConnectorSession session, ConnectorTableHandle tableHan
metastore.renameColumn(hiveTableHandle.getSchemaName(), hiveTableHandle.getTableName(), sourceHandle.getName(), target);
}
+ @Override
+ public void dropColumn(ConnectorSession session, ConnectorTableHandle tableHandle, ColumnHandle column)
+ {
+ HiveTableHandle hiveTableHandle = (HiveTableHandle) tableHandle;
+ HiveColumnHandle columnHandle = (HiveColumnHandle) column;
+
+ metastore.dropColumn(hiveTableHandle.getSchemaName(), hiveTableHandle.getTableName(), columnHandle.getName());
+ }
+
@Override
public void renameTable(ConnectorSession session, ConnectorTableHandle tableHandle, SchemaTableName newTableName)
{
@@ -738,10 +748,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;
@@ -756,7 +765,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;
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSourceFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSourceFactory.java
index efa076474a33..56062ab38642 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSourceFactory.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSourceFactory.java
@@ -32,6 +32,7 @@ Optional extends ConnectorPageSource> createPageSource(
Path path,
long start,
long length,
+ long fileSize,
Properties schema,
List columns,
TupleDomain effectivePredicate,
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSourceProvider.java b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSourceProvider.java
index e6987890b3d7..ae4ffb3df5f1 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSourceProvider.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSourceProvider.java
@@ -116,6 +116,7 @@ private ConnectorPageSource doCreatePageSource(ConnectorSession session, Connect
hiveSplit.getBucketNumber(),
hiveSplit.getStart(),
hiveSplit.getLength(),
+ hiveSplit.getFileSize(),
hiveSplit.getSchema(),
hiveSplit.getEffectivePredicate(),
hiveColumns,
@@ -139,6 +140,7 @@ public static Optional createHivePageSource(
OptionalInt bucketNumber,
long start,
long length,
+ long fileSize,
Properties schema,
TupleDomain effectivePredicate,
List hiveColumns,
@@ -157,6 +159,7 @@ public static Optional createHivePageSource(
path,
start,
length,
+ fileSize,
schema,
extractRegularColumnHandles(regularColumnMappings, true),
effectivePredicate,
@@ -183,6 +186,7 @@ public static Optional createHivePageSource(
path,
start,
length,
+ fileSize,
schema,
extractRegularColumnHandles(regularColumnMappings, doCoercion),
effectivePredicate,
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveRecordCursorProvider.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveRecordCursorProvider.java
index d5fc472e4c9b..68e3cd709b48 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveRecordCursorProvider.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveRecordCursorProvider.java
@@ -34,6 +34,7 @@ Optional createRecordCursor(
Path path,
long start,
long length,
+ long fileSize,
Properties schema,
List columns,
TupleDomain effectivePredicate,
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 9c8f6383dd31..e0968dedecbf 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,6 +33,7 @@ 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 READ_AS_QUERY_USER = "read_as_query_user";
@@ -83,6 +84,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",
@@ -111,7 +117,7 @@ public HiveSessionProperties(HiveClientConfig config)
booleanSessionProperty(
RCFILE_OPTIMIZED_WRITER_VALIDATE,
"Experimental: RCFile: Validate writer files",
- true,
+ config.isRcfileWriterValidate(),
false),
booleanSessionProperty(
STATISTICS_ENABLED,
@@ -160,6 +166,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);
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveSplit.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveSplit.java
index ff1c203f0443..ba297877a573 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveSplit.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveSplit.java
@@ -37,6 +37,7 @@ public class HiveSplit
private final String path;
private final long start;
private final long length;
+ private final long fileSize;
private final Properties schema;
private final List partitionKeys;
private final List addresses;
@@ -57,6 +58,7 @@ public HiveSplit(
@JsonProperty("path") String path,
@JsonProperty("start") long start,
@JsonProperty("length") long length,
+ @JsonProperty("fileSize") long fileSize,
@JsonProperty("schema") Properties schema,
@JsonProperty("partitionKeys") List partitionKeys,
@JsonProperty("addresses") List addresses,
@@ -68,6 +70,7 @@ public HiveSplit(
requireNonNull(clientId, "clientId is null");
checkArgument(start >= 0, "start must be positive");
checkArgument(length >= 0, "length must be positive");
+ checkArgument(fileSize >= 0, "fileSize must be positive");
requireNonNull(database, "database is null");
requireNonNull(table, "table is null");
requireNonNull(partitionName, "partitionName is null");
@@ -86,6 +89,7 @@ public HiveSplit(
this.path = path;
this.start = start;
this.length = length;
+ this.fileSize = fileSize;
this.schema = schema;
this.partitionKeys = ImmutableList.copyOf(partitionKeys);
this.addresses = ImmutableList.copyOf(addresses);
@@ -137,6 +141,12 @@ public long getLength()
return length;
}
+ @JsonProperty
+ public long getFileSize()
+ {
+ return fileSize;
+ }
+
@JsonProperty
public Properties getSchema()
{
@@ -193,6 +203,7 @@ public Object getInfo()
.put("path", path)
.put("start", start)
.put("length", length)
+ .put("fileSize", fileSize)
.put("hosts", addresses)
.put("database", database)
.put("table", table)
@@ -208,6 +219,7 @@ public String toString()
.addValue(path)
.addValue(start)
.addValue(length)
+ .addValue(fileSize)
.addValue(effectivePredicate)
.toString();
}
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 d6bf69f143c8..50abdf6cfa85 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;
@@ -48,10 +47,9 @@
import java.util.function.Function;
import static com.facebook.presto.hive.HiveErrorCode.HIVE_INVALID_METADATA;
-import static com.facebook.presto.hive.HiveErrorCode.HIVE_METASTORE_ERROR;
+import static com.facebook.presto.hive.HiveErrorCode.HIVE_PARTITION_DROPPED_DURING_QUERY;
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_PARTITION_DROPPED_DURING_QUERY, "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 f082ebdc1a45..ecc57a741325 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
@@ -94,6 +94,7 @@
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;
@@ -191,7 +192,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
@@ -235,7 +236,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)) {
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 910ea7333e94..8e60fc63a6ea 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/PrestoHadoopConfiguration.java b/presto-hive/src/main/java/com/facebook/presto/hive/PrestoHadoopConfiguration.java
deleted file mode 100644
index f65d619b34c7..000000000000
--- 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 dd252ef75746..d0948d19de86 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
@@ -93,6 +93,7 @@
import static com.google.common.base.Preconditions.checkState;
import static com.google.common.base.Strings.isNullOrEmpty;
import static com.google.common.base.Strings.nullToEmpty;
+import static com.google.common.base.Throwables.throwIfInstanceOf;
import static com.google.common.collect.Iterables.toArray;
import static io.airlift.units.DataSize.Unit.MEGABYTE;
import static java.lang.Math.max;
@@ -579,7 +580,7 @@ ObjectMetadata getS3ObjectMetadata(Path path)
throw Throwables.propagate(e);
}
catch (Exception e) {
- Throwables.propagateIfInstanceOf(e, IOException.class);
+ throwIfInstanceOf(e, IOException.class);
throw Throwables.propagate(e);
}
}
@@ -830,7 +831,7 @@ public int read(byte[] buffer, int offset, int length)
throw Throwables.propagate(e);
}
catch (Exception e) {
- Throwables.propagateIfInstanceOf(e, IOException.class);
+ throwIfInstanceOf(e, IOException.class);
throw Throwables.propagate(e);
}
}
@@ -918,7 +919,7 @@ private InputStream openStream(Path path, long start)
throw Throwables.propagate(e);
}
catch (Exception e) {
- Throwables.propagateIfInstanceOf(e, IOException.class);
+ throwIfInstanceOf(e, IOException.class);
throw Throwables.propagate(e);
}
}
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/RcFileFileWriter.java b/presto-hive/src/main/java/com/facebook/presto/hive/RcFileFileWriter.java
index f68e80b73ecf..4b7f23f36ac2 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/RcFileFileWriter.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/RcFileFileWriter.java
@@ -31,6 +31,7 @@
import java.io.IOException;
import java.io.OutputStream;
+import java.io.UncheckedIOException;
import java.util.List;
import java.util.Map;
import java.util.Optional;
@@ -108,7 +109,7 @@ public void appendRows(Page dataPage)
try {
rcFileWriter.write(page);
}
- catch (IOException e) {
+ catch (IOException | UncheckedIOException e) {
throw new PrestoException(HIVE_WRITER_DATA_ERROR, e);
}
}
@@ -119,7 +120,7 @@ public void commit()
try {
rcFileWriter.close();
}
- catch (IOException e) {
+ catch (IOException | UncheckedIOException e) {
try {
rollbackAction.call();
}
@@ -135,7 +136,7 @@ public void commit()
rcFileWriter.validate(input);
}
}
- catch (IOException e) {
+ catch (IOException | UncheckedIOException e) {
throw new PrestoException(HIVE_WRITE_VALIDATION_FAILED, 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 3a37c831cbe4..4facd9771f38 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/metastore/BridgingHiveMetastore.java b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/BridgingHiveMetastore.java
index c43c2f0b85bb..b48163d6abd9 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
@@ -37,6 +37,7 @@
import static com.facebook.presto.hive.metastore.MetastoreUtil.toMetastoreApiPartition;
import static com.facebook.presto.hive.metastore.MetastoreUtil.toMetastoreApiPrivilegeGrantInfo;
import static com.facebook.presto.hive.metastore.MetastoreUtil.toMetastoreApiTable;
+import static com.facebook.presto.hive.metastore.MetastoreUtil.verifyCanDropColumn;
import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED;
import static java.util.Objects.requireNonNull;
import static java.util.function.UnaryOperator.identity;
@@ -205,6 +206,20 @@ public void renameColumn(String databaseName, String tableName, String oldColumn
alterTable(databaseName, tableName, table);
}
+ @Override
+ public void dropColumn(String databaseName, String tableName, String columnName)
+ {
+ verifyCanDropColumn(this, databaseName, tableName, columnName);
+ org.apache.hadoop.hive.metastore.api.Table table = delegate.getTable(databaseName, tableName)
+ .orElseThrow(() -> new TableNotFoundException(new SchemaTableName(databaseName, tableName)));
+ for (FieldSchema fieldSchema : table.getSd().getCols()) {
+ if (fieldSchema.getName().equals(columnName)) {
+ table.getSd().getCols().remove(fieldSchema);
+ }
+ }
+ alterTable(databaseName, tableName, table);
+ }
+
private void alterTable(String databaseName, String tableName, org.apache.hadoop.hive.metastore.api.Table table)
{
delegate.alterTable(databaseName, tableName, table);
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/CachingHiveMetastore.java b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/CachingHiveMetastore.java
index 02d95e57afca..13d2b3255a97 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/CachingHiveMetastore.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/CachingHiveMetastore.java
@@ -42,14 +42,13 @@
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.stream.Collectors;
-import java.util.stream.Stream;
-import java.util.stream.StreamSupport;
import static com.facebook.presto.hive.HiveUtil.toPartitionValues;
import static com.google.common.base.MoreObjects.toStringHelper;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.cache.CacheLoader.asyncReloading;
import static com.google.common.collect.Iterables.transform;
+import static com.google.common.collect.Streams.stream;
import static com.google.common.util.concurrent.MoreExecutors.newDirectExecutorService;
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
@@ -434,11 +433,6 @@ private Map> l
return resultMap.build();
}
- private Stream stream(Iterable keys)
- {
- return StreamSupport.stream(keys.spliterator(), false);
- }
-
@Override
public Optional> getAllTables(String databaseName)
{
@@ -571,6 +565,17 @@ public void renameColumn(String databaseName, String tableName, String oldColumn
}
}
+ @Override
+ public void dropColumn(String databaseName, String tableName, String columnName)
+ {
+ try {
+ delegate.dropColumn(databaseName, tableName, columnName);
+ }
+ finally {
+ invalidateTable(databaseName, tableName);
+ }
+ }
+
protected void invalidateTable(String databaseName, String tableName)
{
tableCache.invalidate(new HiveTableName(databaseName, tableName));
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/ExtendedHiveMetastore.java b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/ExtendedHiveMetastore.java
index c130938b0524..98d50c69ad97 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/ExtendedHiveMetastore.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/ExtendedHiveMetastore.java
@@ -59,6 +59,8 @@ public interface ExtendedHiveMetastore
void renameColumn(String databaseName, String tableName, String oldColumnName, String newColumnName);
+ void dropColumn(String databaseName, String tableName, String columnName);
+
Optional getPartition(String databaseName, String tableName, List partitionValues);
Optional> getPartitionNames(String databaseName, String tableName);
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/MetastoreUtil.java b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/MetastoreUtil.java
index 7490414009c1..e523f4ca4aef 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/MetastoreUtil.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/MetastoreUtil.java
@@ -19,6 +19,7 @@
import com.facebook.presto.hive.TableOfflineException;
import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.SchemaTableName;
+import com.facebook.presto.spi.TableNotFoundException;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import org.apache.hadoop.hive.common.FileUtils;
@@ -57,6 +58,7 @@
import static com.facebook.presto.hive.HiveErrorCode.HIVE_INVALID_METADATA;
import static com.facebook.presto.hive.HiveSplitManager.PRESTO_OFFLINE;
import static com.facebook.presto.hive.metastore.HivePrivilegeInfo.parsePrivilege;
+import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Strings.emptyToNull;
import static com.google.common.base.Strings.isNullOrEmpty;
@@ -602,4 +604,17 @@ public static void verifyOnline(SchemaTableName tableName, Optional part
throw new TableOfflineException(tableName, true, prestoOffline);
}
}
+
+ public static void verifyCanDropColumn(ExtendedHiveMetastore metastore, String databaseName, String tableName, String columnName)
+ {
+ Table table = metastore.getTable(databaseName, tableName)
+ .orElseThrow(() -> new TableNotFoundException(new SchemaTableName(databaseName, tableName)));
+
+ if (table.getPartitionColumns().stream().anyMatch(column -> column.getName().equals(columnName))) {
+ throw new PrestoException(NOT_SUPPORTED, "Cannot drop partition columns");
+ }
+ if (table.getDataColumns().size() <= 1) {
+ throw new PrestoException(NOT_SUPPORTED, "Cannot drop the only column in a table");
+ }
+ }
}
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/SemiTransactionalHiveMetastore.java b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/SemiTransactionalHiveMetastore.java
index 94ef07774db8..344addcf895d 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/SemiTransactionalHiveMetastore.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/SemiTransactionalHiveMetastore.java
@@ -349,6 +349,11 @@ public synchronized void renameColumn(String databaseName, String tableName, Str
setExclusive((delegate, hdfsEnvironment) -> delegate.renameColumn(databaseName, tableName, oldColumnName, newColumnName));
}
+ public synchronized void dropColumn(String databaseName, String tableName, String columnName)
+ {
+ setExclusive((delegate, hdfsEnvironment) -> delegate.dropColumn(databaseName, tableName, columnName));
+ }
+
public synchronized void finishInsertIntoExistingTable(ConnectorSession session, String databaseName, String tableName, Path currentLocation, List fileNames)
{
// Data can only be inserted into partitions and unpartitioned tables. They can never be inserted into a partitioned table.
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/file/FileHiveMetastore.java b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/file/FileHiveMetastore.java
index b6b7e3f95dde..74d208bb95be 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/file/FileHiveMetastore.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/file/FileHiveMetastore.java
@@ -67,6 +67,7 @@
import static com.facebook.presto.hive.metastore.Database.DEFAULT_DATABASE_NAME;
import static com.facebook.presto.hive.metastore.HivePrivilegeInfo.HivePrivilege.OWNERSHIP;
import static com.facebook.presto.hive.metastore.MetastoreUtil.makePartName;
+import static com.facebook.presto.hive.metastore.MetastoreUtil.verifyCanDropColumn;
import static com.facebook.presto.hive.metastore.PrincipalType.ROLE;
import static com.facebook.presto.hive.metastore.PrincipalType.USER;
import static com.facebook.presto.spi.StandardErrorCode.ALREADY_EXISTS;
@@ -427,6 +428,27 @@ public synchronized void renameColumn(String databaseName, String tableName, Str
});
}
+ @Override
+ public synchronized void dropColumn(String databaseName, String tableName, String columnName)
+ {
+ alterTable(databaseName, tableName, oldTable -> {
+ verifyCanDropColumn(this, databaseName, tableName, columnName);
+ if (!oldTable.getColumn(columnName).isPresent()) {
+ SchemaTableName name = new SchemaTableName(databaseName, tableName);
+ throw new ColumnNotFoundException(name, columnName);
+ }
+
+ ImmutableList.Builder newDataColumns = ImmutableList.builder();
+ for (Column fieldSchema : oldTable.getDataColumns()) {
+ if (!fieldSchema.getName().equals(columnName)) {
+ newDataColumns.add(fieldSchema);
+ }
+ }
+
+ return oldTable.withDataColumns(newDataColumns.build());
+ });
+ }
+
private void alterTable(String databaseName, String tableName, Function alterFunction)
{
requireNonNull(databaseName, "databaseName is null");
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/orc/DwrfPageSourceFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/orc/DwrfPageSourceFactory.java
index 80ee0f3de20e..d52ead0fda89 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/orc/DwrfPageSourceFactory.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/orc/DwrfPageSourceFactory.java
@@ -35,6 +35,7 @@
import static com.facebook.presto.hive.HiveSessionProperties.getOrcMaxBufferSize;
import static com.facebook.presto.hive.HiveSessionProperties.getOrcMaxMergeDistance;
+import static com.facebook.presto.hive.HiveSessionProperties.getOrcMaxReadBlockSize;
import static com.facebook.presto.hive.HiveSessionProperties.getOrcStreamBufferSize;
import static com.facebook.presto.hive.HiveUtil.isDeserializerClass;
import static com.facebook.presto.hive.orc.OrcPageSourceFactory.createOrcPageSource;
@@ -61,6 +62,7 @@ public Optional extends ConnectorPageSource> createPageSource(Configuration co
Path path,
long start,
long length,
+ long fileSize,
Properties schema,
List columns,
TupleDomain effectivePredicate,
@@ -78,6 +80,7 @@ public Optional extends ConnectorPageSource> createPageSource(Configuration co
path,
start,
length,
+ fileSize,
columns,
false,
effectivePredicate,
@@ -86,6 +89,7 @@ public Optional extends ConnectorPageSource> createPageSource(Configuration co
getOrcMaxMergeDistance(session),
getOrcMaxBufferSize(session),
getOrcStreamBufferSize(session),
+ getOrcMaxReadBlockSize(session),
false,
stats));
}
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcPageSource.java b/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcPageSource.java
index 65c97e73fbf0..a09794c54cb4 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcPageSource.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcPageSource.java
@@ -176,6 +176,7 @@ public void close()
closed = true;
try {
+ stats.addMaxCombinedBytesPerRow(recordReader.getMaxCombinedBytesPerRow());
recordReader.close();
}
catch (IOException e) {
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcPageSourceFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcPageSourceFactory.java
index 55a4cc7502c2..eef13bb68293 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcPageSourceFactory.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcPageSourceFactory.java
@@ -60,6 +60,7 @@
import static com.facebook.presto.hive.HiveErrorCode.HIVE_MISSING_DATA;
import static com.facebook.presto.hive.HiveSessionProperties.getOrcMaxBufferSize;
import static com.facebook.presto.hive.HiveSessionProperties.getOrcMaxMergeDistance;
+import static com.facebook.presto.hive.HiveSessionProperties.getOrcMaxReadBlockSize;
import static com.facebook.presto.hive.HiveSessionProperties.getOrcStreamBufferSize;
import static com.facebook.presto.hive.HiveSessionProperties.isOrcBloomFiltersEnabled;
import static com.facebook.presto.hive.HiveUtil.isDeserializerClass;
@@ -97,6 +98,7 @@ public Optional extends ConnectorPageSource> createPageSource(
Path path,
long start,
long length,
+ long fileSize,
Properties schema,
List columns,
TupleDomain effectivePredicate,
@@ -114,6 +116,7 @@ public Optional extends ConnectorPageSource> createPageSource(
path,
start,
length,
+ fileSize,
columns,
useOrcColumnNames,
effectivePredicate,
@@ -122,6 +125,7 @@ public Optional extends ConnectorPageSource> createPageSource(
getOrcMaxMergeDistance(session),
getOrcMaxBufferSize(session),
getOrcStreamBufferSize(session),
+ getOrcMaxReadBlockSize(session),
isOrcBloomFiltersEnabled(session),
stats));
}
@@ -134,6 +138,7 @@ public static OrcPageSource createOrcPageSource(
Path path,
long start,
long length,
+ long fileSize,
List columns,
boolean useOrcColumnNames,
TupleDomain effectivePredicate,
@@ -142,15 +147,15 @@ public static OrcPageSource createOrcPageSource(
DataSize maxMergeDistance,
DataSize maxBufferSize,
DataSize streamBufferSize,
+ DataSize maxReadBlockSize,
boolean orcBloomFiltersEnabled,
FileFormatDataSourceStats stats)
{
OrcDataSource orcDataSource;
try {
FileSystem fileSystem = hdfsEnvironment.getFileSystem(sessionUser, path, configuration);
- long size = fileSystem.getFileStatus(path).getLen();
FSDataInputStream inputStream = fileSystem.open(path);
- orcDataSource = new HdfsOrcDataSource(new OrcDataSourceId(path.toString()), size, maxMergeDistance, maxBufferSize, streamBufferSize, inputStream, stats);
+ orcDataSource = new HdfsOrcDataSource(new OrcDataSourceId(path.toString()), fileSize, maxMergeDistance, maxBufferSize, streamBufferSize, inputStream, stats);
}
catch (Exception e) {
if (nullToEmpty(e.getMessage()).trim().equals("Filesystem closed") ||
@@ -162,7 +167,7 @@ public static OrcPageSource createOrcPageSource(
AggregatedMemoryContext systemMemoryUsage = new AggregatedMemoryContext();
try {
- OrcReader reader = new OrcReader(orcDataSource, metadataReader, maxMergeDistance, maxBufferSize);
+ OrcReader reader = new OrcReader(orcDataSource, metadataReader, maxMergeDistance, maxBufferSize, maxReadBlockSize);
List physicalColumns = getPhysicalHiveColumnHandles(columns, useOrcColumnNames, reader, path);
ImmutableMap.Builder includedColumns = ImmutableMap.builder();
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/HdfsParquetDataSource.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/HdfsParquetDataSource.java
index 6fbb272d66f4..1b35375d0250 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/HdfsParquetDataSource.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/HdfsParquetDataSource.java
@@ -90,12 +90,11 @@ private void readInternal(long position, byte[] buffer, int bufferOffset, int bu
}
}
- public static HdfsParquetDataSource buildHdfsParquetDataSource(FileSystem fileSystem, Path path, long start, long length)
+ public static HdfsParquetDataSource buildHdfsParquetDataSource(FileSystem fileSystem, Path path, long start, long length, long fileSize)
{
try {
- long size = fileSystem.getFileStatus(path).getLen();
FSDataInputStream inputStream = fileSystem.open(path);
- return new HdfsParquetDataSource(path, size, inputStream);
+ return new HdfsParquetDataSource(path, fileSize, inputStream);
}
catch (Exception e) {
if (nullToEmpty(e.getMessage()).trim().equals("Filesystem closed") ||
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetHiveRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetHiveRecordCursor.java
index a2c4ae8801b1..b7d6de8b219d 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetHiveRecordCursor.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetHiveRecordCursor.java
@@ -87,6 +87,7 @@
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 com.google.common.base.Throwables.throwIfInstanceOf;
import static io.airlift.slice.Slices.wrappedBuffer;
import static java.lang.Float.floatToRawIntBits;
import static java.lang.Math.max;
@@ -123,6 +124,7 @@ public ParquetHiveRecordCursor(
Path path,
long start,
long length,
+ long fileSize,
Properties splitSchema,
List columns,
boolean useParquetColumnNames,
@@ -162,6 +164,7 @@ public ParquetHiveRecordCursor(
path,
start,
length,
+ fileSize,
columns,
useParquetColumnNames,
predicatePushdownEnabled,
@@ -319,6 +322,7 @@ private ParquetRecordReader createParquetRecordReader(
Path path,
long start,
long length,
+ long fileSize,
List columns,
boolean useParquetColumnNames,
boolean predicatePushdownEnabled,
@@ -327,7 +331,7 @@ private ParquetRecordReader createParquetRecordReader(
ParquetDataSource dataSource = null;
try {
FileSystem fileSystem = hdfsEnvironment.getFileSystem(sessionUser, path, configuration);
- dataSource = buildHdfsParquetDataSource(fileSystem, path, start, length);
+ dataSource = buildHdfsParquetDataSource(fileSystem, path, start, length, fileSize);
ParquetMetadata parquetMetadata = hdfsEnvironment.doAs(sessionUser, () -> ParquetFileReader.readFooter(configuration, path, NO_FILTER));
List blocks = parquetMetadata.getBlocks();
FileMetaData fileMetaData = parquetMetadata.getFileMetaData();
@@ -371,7 +375,7 @@ private ParquetRecordReader createParquetRecordReader(
});
}
catch (Exception e) {
- Throwables.propagateIfInstanceOf(e, PrestoException.class);
+ throwIfInstanceOf(e, PrestoException.class);
if (e instanceof InterruptedException) {
Thread.currentThread().interrupt();
throw Throwables.propagate(e);
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSourceFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSourceFactory.java
index 9990227d2ef8..edbdf37dd04b 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSourceFactory.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSourceFactory.java
@@ -94,6 +94,7 @@ public Optional extends ConnectorPageSource> createPageSource(
Path path,
long start,
long length,
+ long fileSize,
Properties schema,
List columns,
TupleDomain effectivePredicate,
@@ -114,6 +115,7 @@ public Optional extends ConnectorPageSource> createPageSource(
path,
start,
length,
+ fileSize,
schema,
columns,
useParquetColumnNames,
@@ -129,6 +131,7 @@ public static ParquetPageSource createParquetPageSource(
Path path,
long start,
long length,
+ long fileSize,
Properties schema,
List columns,
boolean useParquetColumnNames,
@@ -141,8 +144,8 @@ public static ParquetPageSource createParquetPageSource(
ParquetDataSource dataSource = null;
try {
FileSystem fileSystem = hdfsEnvironment.getFileSystem(user, path, configuration);
- dataSource = buildHdfsParquetDataSource(fileSystem, path, start, length);
- ParquetMetadata parquetMetadata = ParquetMetadataReader.readFooter(fileSystem, path);
+ dataSource = buildHdfsParquetDataSource(fileSystem, path, start, length, fileSize);
+ ParquetMetadata parquetMetadata = ParquetMetadataReader.readFooter(fileSystem, path, fileSize);
FileMetaData fileMetaData = parquetMetadata.getFileMetaData();
MessageType fileSchema = fileMetaData.getSchema();
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetRecordCursorProvider.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetRecordCursorProvider.java
index 14f45946b392..056ff1269e2b 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetRecordCursorProvider.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetRecordCursorProvider.java
@@ -68,6 +68,7 @@ public Optional createRecordCursor(
Path path,
long start,
long length,
+ long fileSize,
Properties schema,
List columns,
TupleDomain effectivePredicate,
@@ -85,6 +86,7 @@ public Optional createRecordCursor(
path,
start,
length,
+ fileSize,
schema,
columns,
useParquetColumnNames,
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetMetadataReader.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetMetadataReader.java
index 1b9a249f0f9a..c080f30fda1d 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetMetadataReader.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetMetadataReader.java
@@ -14,7 +14,6 @@
package com.facebook.presto.hive.parquet.reader;
import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import parquet.format.ColumnChunk;
@@ -62,10 +61,9 @@ public final class ParquetMetadataReader
private ParquetMetadataReader() {}
- public static ParquetMetadata readFooter(FileSystem fileSystem, Path file)
+ public static ParquetMetadata readFooter(FileSystem fileSystem, Path file, long fileSize)
throws IOException
{
- FileStatus fileStatus = fileSystem.getFileStatus(file);
try (FSDataInputStream inputStream = fileSystem.open(file)) {
// Parquet File Layout:
//
@@ -75,9 +73,8 @@ public static ParquetMetadata readFooter(FileSystem fileSystem, Path file)
// 4 bytes: MetadataLength
// MAGIC
- long length = fileStatus.getLen();
- validateParquet(length >= MAGIC.length + PARQUET_METADATA_LENGTH + MAGIC.length, "%s is not a valid Parquet File", file);
- long metadataLengthIndex = length - PARQUET_METADATA_LENGTH - MAGIC.length;
+ validateParquet(fileSize >= MAGIC.length + PARQUET_METADATA_LENGTH + MAGIC.length, "%s is not a valid Parquet File", file);
+ long metadataLengthIndex = fileSize - PARQUET_METADATA_LENGTH - MAGIC.length;
inputStream.seek(metadataLengthIndex);
int metadataLength = readIntLittleEndian(inputStream);
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetReader.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetReader.java
index d0199f546cf9..7ac5883045e6 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetReader.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetReader.java
@@ -22,6 +22,7 @@
import com.facebook.presto.spi.block.Block;
import com.facebook.presto.spi.block.InterleavedBlock;
import com.facebook.presto.spi.block.RunLengthEncodedBlock;
+import com.facebook.presto.spi.type.MapType;
import com.facebook.presto.spi.type.NamedTypeSignature;
import com.facebook.presto.spi.type.Type;
import com.facebook.presto.spi.type.TypeManager;
@@ -203,14 +204,13 @@ private Block readMap(Type type, List path, IntList elementOffsets)
}
return RunLengthEncodedBlock.create(parameters.get(0), null, batchSize);
}
- InterleavedBlock interleavedBlock = new InterleavedBlock(new Block[] {blocks[0], blocks[1]});
int[] offsets = new int[batchSize + 1];
for (int i = 1; i < offsets.length; i++) {
- int elementPositionCount = keyOffsets.getInt(i - 1) * 2;
- elementOffsets.add(elementPositionCount);
+ int elementPositionCount = keyOffsets.getInt(i - 1);
+ elementOffsets.add(elementPositionCount * 2);
offsets[i] = offsets[i - 1] + elementPositionCount;
}
- return new ArrayBlock(batchSize, new boolean[batchSize], offsets, interleavedBlock);
+ return ((MapType) type).createBlockFromKeyValue(new boolean[batchSize], offsets, blocks[0], blocks[1]);
}
public Block readStruct(Type type, List path)
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetShortDecimalColumnReader.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetShortDecimalColumnReader.java
index b96dd1efe59e..b47d42115baa 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetShortDecimalColumnReader.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetShortDecimalColumnReader.java
@@ -59,7 +59,15 @@ else if (columnDescriptor.getType().equals(INT64)) {
protected void skipValue()
{
if (definitionLevel == columnDescriptor.getMaxDefinitionLevel()) {
- valuesReader.readBytes();
+ if (columnDescriptor.getType().equals(INT32)) {
+ valuesReader.readInteger();
+ }
+ else if (columnDescriptor.getType().equals(INT64)) {
+ valuesReader.readLong();
+ }
+ else {
+ valuesReader.readBytes();
+ }
}
}
}
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/rcfile/RcFilePageSourceFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/rcfile/RcFilePageSourceFactory.java
index ec845329b0f7..05b3adbd2459 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/rcfile/RcFilePageSourceFactory.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/rcfile/RcFilePageSourceFactory.java
@@ -94,6 +94,7 @@ public Optional extends ConnectorPageSource> createPageSource(
Path path,
long start,
long length,
+ long fileSize,
Properties schema,
List columns,
TupleDomain effectivePredicate,
@@ -111,11 +112,9 @@ else if (deserializerClassName.equals(ColumnarSerDe.class.getName())) {
return Optional.empty();
}
- long size;
FSDataInputStream inputStream;
try {
FileSystem fileSystem = hdfsEnvironment.getFileSystem(session.getUser(), path, configuration);
- size = fileSystem.getFileStatus(path).getLen();
inputStream = fileSystem.open(path);
}
catch (Exception e) {
@@ -133,7 +132,7 @@ else if (deserializerClassName.equals(ColumnarSerDe.class.getName())) {
}
RcFileReader rcFileReader = new RcFileReader(
- new HdfsRcFileDataSource(path.toString(), inputStream, size, stats),
+ new HdfsRcFileDataSource(path.toString(), inputStream, fileSize, stats),
rcFileEncoding,
readColumns.build(),
new AircompressorCodecFactory(new HadoopCodecFactory(configuration.getClassLoader())),
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/security/LegacyAccessControl.java b/presto-hive/src/main/java/com/facebook/presto/hive/security/LegacyAccessControl.java
index 731b96e01e63..788ec92c740b 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/security/LegacyAccessControl.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/security/LegacyAccessControl.java
@@ -29,6 +29,7 @@
import java.util.function.Function;
import static com.facebook.presto.spi.security.AccessDeniedException.denyAddColumn;
+import static com.facebook.presto.spi.security.AccessDeniedException.denyDropColumn;
import static com.facebook.presto.spi.security.AccessDeniedException.denyDropTable;
import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameColumn;
import static com.facebook.presto.spi.security.AccessDeniedException.denyRenameTable;
@@ -41,6 +42,7 @@ public class LegacyAccessControl
private final boolean allowDropTable;
private final boolean allowRenameTable;
private final boolean allowAddColumn;
+ private final boolean allowDropColumn;
private final boolean allowRenameColumn;
@Inject
@@ -54,6 +56,7 @@ public LegacyAccessControl(
allowDropTable = securityConfig.getAllowDropTable();
allowRenameTable = securityConfig.getAllowRenameTable();
allowAddColumn = securityConfig.getAllowAddColumn();
+ allowDropColumn = securityConfig.getAllowDropColumn();
allowRenameColumn = securityConfig.getAllowRenameColumn();
}
@@ -133,6 +136,14 @@ public void checkCanAddColumn(ConnectorTransactionHandle transaction, Identity i
}
}
+ @Override
+ public void checkCanDropColumn(ConnectorTransactionHandle transactionHandle, Identity identity, SchemaTableName tableName)
+ {
+ if (!allowDropColumn) {
+ denyDropColumn(tableName.toString());
+ }
+ }
+
@Override
public void checkCanRenameColumn(ConnectorTransactionHandle transaction, Identity identity, SchemaTableName tableName)
{
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/security/LegacySecurityConfig.java b/presto-hive/src/main/java/com/facebook/presto/hive/security/LegacySecurityConfig.java
index 72c3ac414073..c1573809deda 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/security/LegacySecurityConfig.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/security/LegacySecurityConfig.java
@@ -19,6 +19,7 @@
public class LegacySecurityConfig
{
private boolean allowAddColumn;
+ private boolean allowDropColumn;
private boolean allowDropTable;
private boolean allowRenameTable;
private boolean allowRenameColumn;
@@ -36,6 +37,19 @@ public LegacySecurityConfig setAllowAddColumn(boolean allowAddColumn)
return this;
}
+ public boolean getAllowDropColumn()
+ {
+ return this.allowDropColumn;
+ }
+
+ @Config("hive.allow-drop-column")
+ @ConfigDescription("Allow Hive connector to drop column")
+ public LegacySecurityConfig setAllowDropColumn(boolean allowDropColumn)
+ {
+ this.allowDropColumn = allowDropColumn;
+ return this;
+ }
+
public boolean getAllowDropTable()
{
return this.allowDropTable;
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/security/SqlStandardAccessControl.java b/presto-hive/src/main/java/com/facebook/presto/hive/security/SqlStandardAccessControl.java
index 8b0fa940109c..fee66951bd9d 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/security/SqlStandardAccessControl.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/security/SqlStandardAccessControl.java
@@ -43,6 +43,7 @@
import static com.facebook.presto.spi.security.AccessDeniedException.denyCreateView;
import static com.facebook.presto.spi.security.AccessDeniedException.denyCreateViewWithSelect;
import static com.facebook.presto.spi.security.AccessDeniedException.denyDeleteTable;
+import static com.facebook.presto.spi.security.AccessDeniedException.denyDropColumn;
import static com.facebook.presto.spi.security.AccessDeniedException.denyDropSchema;
import static com.facebook.presto.spi.security.AccessDeniedException.denyDropTable;
import static com.facebook.presto.spi.security.AccessDeniedException.denyDropView;
@@ -159,6 +160,14 @@ public void checkCanAddColumn(ConnectorTransactionHandle transaction, Identity i
}
}
+ @Override
+ public void checkCanDropColumn(ConnectorTransactionHandle transaction, Identity identity, SchemaTableName tableName)
+ {
+ if (!checkTablePermission(transaction, identity, tableName, OWNERSHIP)) {
+ denyDropColumn(tableName.toString());
+ }
+ }
+
@Override
public void checkCanRenameColumn(ConnectorTransactionHandle transaction, Identity identity, SchemaTableName tableName)
{
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/thrift/Transport.java b/presto-hive/src/main/java/com/facebook/presto/hive/thrift/Transport.java
index b9bdd521bdae..a8ffe3df7019 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/thrift/Transport.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/thrift/Transport.java
@@ -82,7 +82,7 @@ private static void closeQuietly(Closeable closeable)
private static Socket createSocksSocket(HostAndPort proxy)
{
- SocketAddress address = InetSocketAddress.createUnresolved(proxy.getHostText(), proxy.getPort());
+ SocketAddress address = InetSocketAddress.createUnresolved(proxy.getHost(), proxy.getPort());
return new Socket(new Proxy(Proxy.Type.SOCKS, address));
}
diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/util/ConfigurationUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/util/ConfigurationUtils.java
index 27d9d0854c3e..b647ab89791c 100644
--- a/presto-hive/src/main/java/com/facebook/presto/hive/util/ConfigurationUtils.java
+++ b/presto-hive/src/main/java/com/facebook/presto/hive/util/ConfigurationUtils.java
@@ -14,6 +14,7 @@
package com.facebook.presto.hive.util;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.JobConf;
import java.util.Map;
@@ -27,4 +28,12 @@ public static void copy(Configuration from, Configuration to)
to.set(entry.getKey(), entry.getValue());
}
}
+
+ public static JobConf toJobConf(Configuration conf)
+ {
+ if (conf instanceof JobConf) {
+ return (JobConf) conf;
+ }
+ return new JobConf(conf);
+ }
}
diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursorProvider.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursorProvider.java
index ed2cf1b9defd..9373a4c5232c 100644
--- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursorProvider.java
+++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursorProvider.java
@@ -68,6 +68,7 @@ public Optional createRecordCursor(
Path path,
long start,
long length,
+ long fileSize,
Properties schema,
List columns,
TupleDomain effectivePredicate,
diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java b/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java
index 4526d4e215ee..3cc69ce64418 100644
--- a/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java
+++ b/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java
@@ -66,6 +66,8 @@
import com.facebook.presto.spi.predicate.Range;
import com.facebook.presto.spi.predicate.TupleDomain;
import com.facebook.presto.spi.predicate.ValueSet;
+import com.facebook.presto.spi.type.ArrayType;
+import com.facebook.presto.spi.type.MapType;
import com.facebook.presto.spi.type.NamedTypeSignature;
import com.facebook.presto.spi.type.SqlDate;
import com.facebook.presto.spi.type.SqlTimestamp;
@@ -78,8 +80,6 @@
import com.facebook.presto.testing.MaterializedRow;
import com.facebook.presto.testing.TestingConnectorSession;
import com.facebook.presto.testing.TestingNodeManager;
-import com.facebook.presto.type.ArrayType;
-import com.facebook.presto.type.MapType;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableMultimap;
@@ -124,7 +124,6 @@
import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.PARTITION_KEY;
import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR;
import static com.facebook.presto.hive.HiveErrorCode.HIVE_INVALID_PARTITION_VALUE;
-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.HiveMetadata.PRESTO_QUERY_ID_NAME;
import static com.facebook.presto.hive.HiveMetadata.PRESTO_VERSION_NAME;
@@ -2155,7 +2154,7 @@ private void doInsertIntoNewPartition(HiveStorageFormat storageFormat, SchemaTab
try (Transaction transaction = newTransaction()) {
// verify partitions were created
List partitionNames = transaction.getMetastore(tableName.getSchemaName()).getPartitionNames(tableName.getSchemaName(), tableName.getTableName())
- .orElseThrow(() -> new PrestoException(HIVE_METASTORE_ERROR, "Partition metadata not available"));
+ .orElseThrow(() -> new AssertionError("Table does not exist: " + tableName));
assertEqualsIgnoreOrder(partitionNames, CREATE_TABLE_PARTITIONED_DATA.getMaterializedRows().stream()
.map(row -> "ds=" + row.getField(CREATE_TABLE_PARTITIONED_DATA.getTypes().size() - 1))
.collect(toList()));
@@ -2272,7 +2271,7 @@ private void doInsertIntoExistingPartition(HiveStorageFormat storageFormat, Sche
// verify partitions were created
List partitionNames = transaction.getMetastore(tableName.getSchemaName()).getPartitionNames(tableName.getSchemaName(), tableName.getTableName())
- .orElseThrow(() -> new PrestoException(HIVE_METASTORE_ERROR, "Partition metadata not available"));
+ .orElseThrow(() -> new AssertionError("Table does not exist: " + tableName));
assertEqualsIgnoreOrder(partitionNames, CREATE_TABLE_PARTITIONED_DATA.getMaterializedRows().stream()
.map(row -> "ds=" + row.getField(CREATE_TABLE_PARTITIONED_DATA.getTypes().size() - 1))
.collect(toList()));
@@ -2395,7 +2394,7 @@ private void doTestMetadataDelete(HiveStorageFormat storageFormat, SchemaTableNa
// verify partitions were created
List partitionNames = transaction.getMetastore(tableName.getSchemaName()).getPartitionNames(tableName.getSchemaName(), tableName.getTableName())
- .orElseThrow(() -> new PrestoException(HIVE_METASTORE_ERROR, "Partition metadata not available"));
+ .orElseThrow(() -> new AssertionError("Table does not exist: " + tableName));
assertEqualsIgnoreOrder(partitionNames, CREATE_TABLE_PARTITIONED_DATA.getMaterializedRows().stream()
.map(row -> "ds=" + row.getField(CREATE_TABLE_PARTITIONED_DATA.getTypes().size() - 1))
.collect(toList()));
@@ -3253,7 +3252,7 @@ private void doTestTransactionDeleteInsert(
// verify partitions
List partitionNames = transaction.getMetastore(tableName.getSchemaName())
.getPartitionNames(tableName.getSchemaName(), tableName.getTableName())
- .orElseThrow(() -> new PrestoException(HIVE_METASTORE_ERROR, "Partition metadata not available"));
+ .orElseThrow(() -> new AssertionError("Table does not exist: " + tableName));
assertEqualsIgnoreOrder(
partitionNames,
expectedData.getMaterializedRows().stream()
diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveFileFormats.java b/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveFileFormats.java
index 2b132c688354..ea56b623f34d 100644
--- a/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveFileFormats.java
+++ b/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveFileFormats.java
@@ -23,10 +23,12 @@
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.type.ArrayType;
import com.facebook.presto.spi.type.CharType;
import com.facebook.presto.spi.type.DateType;
import com.facebook.presto.spi.type.DecimalType;
import com.facebook.presto.spi.type.Decimals;
+import com.facebook.presto.spi.type.RowType;
import com.facebook.presto.spi.type.SqlDate;
import com.facebook.presto.spi.type.SqlDecimal;
import com.facebook.presto.spi.type.SqlTimestamp;
@@ -36,8 +38,6 @@
import com.facebook.presto.testing.MaterializedResult;
import com.facebook.presto.testing.MaterializedRow;
import com.facebook.presto.tests.StructuralTestUtil;
-import com.facebook.presto.type.ArrayType;
-import com.facebook.presto.type.RowType;
import com.google.common.base.Joiner;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/HiveTestUtils.java b/presto-hive/src/test/java/com/facebook/presto/hive/HiveTestUtils.java
index 8fa624c8e28e..0c507cfd6835 100644
--- a/presto-hive/src/test/java/com/facebook/presto/hive/HiveTestUtils.java
+++ b/presto-hive/src/test/java/com/facebook/presto/hive/HiveTestUtils.java
@@ -23,12 +23,12 @@
import com.facebook.presto.metadata.FunctionRegistry;
import com.facebook.presto.spi.ColumnHandle;
import com.facebook.presto.spi.ConnectorSession;
+import com.facebook.presto.spi.type.MapType;
import com.facebook.presto.spi.type.StandardTypes;
import com.facebook.presto.spi.type.Type;
import com.facebook.presto.spi.type.TypeSignatureParameter;
import com.facebook.presto.sql.analyzer.FeaturesConfig;
import com.facebook.presto.testing.TestingConnectorSession;
-import com.facebook.presto.type.MapType;
import com.facebook.presto.type.TypeRegistry;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveClientConfig.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveClientConfig.java
index 37bca3e51553..b7e90a43c3c7 100644
--- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveClientConfig.java
+++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveClientConfig.java
@@ -80,7 +80,9 @@ public void testDefaults()
.setOrcMaxMergeDistance(new DataSize(1, Unit.MEGABYTE))
.setOrcMaxBufferSize(new DataSize(8, Unit.MEGABYTE))
.setOrcStreamBufferSize(new DataSize(8, Unit.MEGABYTE))
- .setRcfileOptimizedWriterEnabled(false)
+ .setOrcMaxReadBlockSize(new DataSize(16, Unit.MEGABYTE))
+ .setRcfileOptimizedWriterEnabled(true)
+ .setRcfileWriterValidate(false)
.setHiveMetastoreAuthenticationType(HiveMetastoreAuthenticationType.NONE)
.setHdfsAuthenticationType(HdfsAuthenticationType.NONE)
.setHdfsImpersonationEnabled(false)
@@ -139,7 +141,9 @@ public void testExplicitPropertyMappings()
.put("hive.orc.max-merge-distance", "22kB")
.put("hive.orc.max-buffer-size", "44kB")
.put("hive.orc.stream-buffer-size", "55kB")
- .put("hive.rcfile-optimized-writer.enabled", "true")
+ .put("hive.orc.max-read-block-size", "66kB")
+ .put("hive.rcfile-optimized-writer.enabled", "false")
+ .put("hive.rcfile.writer.validate", "true")
.put("hive.metastore.authentication.type", "KERBEROS")
.put("hive.hdfs.authentication.type", "KERBEROS")
.put("hive.hdfs.impersonation.enabled", "true")
@@ -195,7 +199,9 @@ public void testExplicitPropertyMappings()
.setOrcMaxMergeDistance(new DataSize(22, Unit.KILOBYTE))
.setOrcMaxBufferSize(new DataSize(44, Unit.KILOBYTE))
.setOrcStreamBufferSize(new DataSize(55, Unit.KILOBYTE))
- .setRcfileOptimizedWriterEnabled(true)
+ .setOrcMaxReadBlockSize(new DataSize(66, Unit.KILOBYTE))
+ .setRcfileOptimizedWriterEnabled(false)
+ .setRcfileWriterValidate(true)
.setHiveMetastoreAuthenticationType(HiveMetastoreAuthenticationType.KERBEROS)
.setHdfsAuthenticationType(HdfsAuthenticationType.KERBEROS)
.setHdfsImpersonationEnabled(true)
diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveFileFormats.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveFileFormats.java
index 8aaac9350b64..ba9633c73d0a 100644
--- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveFileFormats.java
+++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveFileFormats.java
@@ -24,12 +24,12 @@
import com.facebook.presto.spi.RecordCursor;
import com.facebook.presto.spi.RecordPageSource;
import com.facebook.presto.spi.predicate.TupleDomain;
+import com.facebook.presto.spi.type.ArrayType;
+import com.facebook.presto.spi.type.RowType;
import com.facebook.presto.testing.TestingConnectorSession;
import com.facebook.presto.twitter.hive.thrift.HiveThriftFieldIdResolverFactory;
import com.facebook.presto.twitter.hive.thrift.ThriftGenericRow;
import com.facebook.presto.twitter.hive.thrift.ThriftHiveRecordCursorProvider;
-import com.facebook.presto.type.ArrayType;
-import com.facebook.presto.type.RowType;
import com.google.common.base.Joiner;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
@@ -698,6 +698,7 @@ private void testCursorProvider(HiveRecordCursorProvider cursorProvider,
OptionalInt.empty(),
split.getStart(),
split.getLength(),
+ split.getLength(),
splitProperties,
TupleDomain.all(),
getColumnHandles(testColumns),
@@ -742,6 +743,7 @@ private void testPageSourceFactory(HivePageSourceFactory sourceFactory,
OptionalInt.empty(),
split.getStart(),
split.getLength(),
+ split.getLength(),
splitProperties,
TupleDomain.all(),
columnHandles,
diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveIntegrationSmokeTest.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveIntegrationSmokeTest.java
index a6eb49489530..1e18af8d72ac 100644
--- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveIntegrationSmokeTest.java
+++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveIntegrationSmokeTest.java
@@ -1861,6 +1861,29 @@ public void testRenameColumn()
assertUpdate("DROP TABLE test_rename_column");
}
+ @Test
+ public void testDropColumn()
+ throws Exception
+ {
+ @Language("SQL") String createTable = "" +
+ "CREATE TABLE test_drop_column\n" +
+ "WITH (\n" +
+ " partitioned_by = ARRAY ['orderstatus']\n" +
+ ")\n" +
+ "AS\n" +
+ "SELECT custkey, orderkey, orderstatus FROM orders";
+
+ assertUpdate(createTable, "SELECT count(*) FROM orders");
+ assertQuery("SELECT orderkey, orderstatus FROM test_drop_column", "SELECT orderkey, orderstatus FROM orders");
+
+ assertQueryFails("ALTER TABLE test_drop_column DROP COLUMN orderstatus", "Cannot drop partition columns");
+ assertUpdate("ALTER TABLE test_drop_column DROP COLUMN orderkey");
+ assertQueryFails("ALTER TABLE test_drop_column DROP COLUMN custkey", "Cannot drop the only column in a table");
+ assertQuery("SELECT * FROM test_drop_column", "SELECT custkey, orderstatus FROM orders");
+
+ assertUpdate("DROP TABLE test_drop_column");
+ }
+
@Test
public void testAvroTypeValidation()
{
diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHivePageSink.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHivePageSink.java
index 3b43955a88d2..508b4e3ac4be 100644
--- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHivePageSink.java
+++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHivePageSink.java
@@ -209,7 +209,7 @@ private static ConnectorPageSource createPageSource(HiveTransactionHandle transa
splitProperties.setProperty(SERIALIZATION_LIB, config.getHiveStorageFormat().getSerDe());
splitProperties.setProperty("columns", Joiner.on(',').join(getColumnHandles().stream().map(HiveColumnHandle::getName).collect(toList())));
splitProperties.setProperty("columns.types", Joiner.on(',').join(getColumnHandles().stream().map(HiveColumnHandle::getHiveType).map(HiveType::getHiveTypeName).collect(toList())));
- HiveSplit split = new HiveSplit(CLIENT_ID, SCHEMA_NAME, TABLE_NAME, "", "file:///" + outputFile.getAbsolutePath(), 0, outputFile.length(), splitProperties, ImmutableList.of(), ImmutableList.of(), OptionalInt.empty(), false, TupleDomain.all(), ImmutableMap.of());
+ HiveSplit split = new HiveSplit(CLIENT_ID, SCHEMA_NAME, TABLE_NAME, "", "file:///" + outputFile.getAbsolutePath(), 0, outputFile.length(), outputFile.length(), splitProperties, ImmutableList.of(), ImmutableList.of(), OptionalInt.empty(), false, TupleDomain.all(), ImmutableMap.of());
HivePageSourceProvider provider = new HivePageSourceProvider(config, createTestHdfsEnvironment(config), getDefaultHiveRecordCursorProvider(config), getDefaultHiveDataStreamFactories(config), TYPE_MANAGER);
return provider.createPageSource(transaction, getSession(config), split, ImmutableList.copyOf(getColumnHandles()));
}
diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveSplit.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveSplit.java
index 9b9ec00dbdfb..fcae1dee939c 100644
--- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveSplit.java
+++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveSplit.java
@@ -47,6 +47,7 @@ public void testJsonRoundTrip()
"path",
42,
88,
+ 88,
schema,
partitionKeys,
addresses,
@@ -65,6 +66,7 @@ public void testJsonRoundTrip()
assertEquals(actual.getPath(), expected.getPath());
assertEquals(actual.getStart(), expected.getStart());
assertEquals(actual.getLength(), expected.getLength());
+ assertEquals(actual.getFileSize(), expected.getFileSize());
assertEquals(actual.getSchema(), expected.getSchema());
assertEquals(actual.getPartitionKeys(), expected.getPartitionKeys());
assertEquals(actual.getAddresses(), expected.getAddresses());
diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestOrcPageSourceMemoryTracking.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestOrcPageSourceMemoryTracking.java
index 771f07b0f6a7..f2fbd7b518cb 100644
--- a/presto-hive/src/test/java/com/facebook/presto/hive/TestOrcPageSourceMemoryTracking.java
+++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestOrcPageSourceMemoryTracking.java
@@ -25,6 +25,7 @@
import com.facebook.presto.operator.project.PageProcessor;
import com.facebook.presto.spi.ColumnHandle;
import com.facebook.presto.spi.ConnectorPageSource;
+import com.facebook.presto.spi.ConnectorSession;
import com.facebook.presto.spi.Page;
import com.facebook.presto.spi.block.Block;
import com.facebook.presto.spi.classloader.ThreadContextClassLoader;
@@ -33,6 +34,7 @@
import com.facebook.presto.sql.gen.ExpressionCompiler;
import com.facebook.presto.sql.planner.plan.PlanNodeId;
import com.facebook.presto.sql.relational.RowExpression;
+import com.facebook.presto.testing.TestingConnectorSession;
import com.facebook.presto.testing.TestingSplit;
import com.facebook.presto.testing.TestingTransactionHandle;
import com.google.common.base.Joiner;
@@ -41,6 +43,8 @@
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import io.airlift.slice.Slice;
+import io.airlift.stats.Distribution;
+import io.airlift.units.DataSize;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -67,6 +71,7 @@
import org.joda.time.DateTimeZone;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
+import org.testng.annotations.DataProvider;
import org.testng.annotations.Test;
import java.io.File;
@@ -74,6 +79,7 @@
import java.lang.reflect.Constructor;
import java.lang.reflect.Field;
import java.lang.reflect.Method;
+import java.util.Arrays;
import java.util.List;
import java.util.Optional;
import java.util.OptionalInt;
@@ -89,6 +95,7 @@
import static com.facebook.presto.hive.HiveTestUtils.SESSION;
import static com.facebook.presto.hive.HiveTestUtils.TYPE_MANAGER;
import static com.facebook.presto.metadata.MetadataManager.createTestMetadataManager;
+import static com.facebook.presto.orc.OrcReader.MAX_BATCH_SIZE;
import static com.facebook.presto.spi.type.VarcharType.createUnboundedVarcharType;
import static com.facebook.presto.sql.relational.Expressions.field;
import static com.facebook.presto.testing.TestingSession.testSessionBuilder;
@@ -98,6 +105,7 @@
import static com.google.common.collect.Iterables.transform;
import static io.airlift.concurrent.Threads.daemonThreadsNamed;
import static io.airlift.testing.Assertions.assertBetweenInclusive;
+import static io.airlift.units.DataSize.Unit.BYTE;
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.Executors.newCachedThreadPool;
import static java.util.stream.Collectors.toList;
@@ -132,6 +140,12 @@ public class TestOrcPageSourceMemoryTracking
private File tempFile;
private TestPreparer testPreparer;
+ @DataProvider(name = "rowCount")
+ public static Object[][] rowCount()
+ {
+ return new Object[][] { { 50_000 }, { 10_000 }, { 5_000 } };
+ }
+
@BeforeClass
public void setUp()
throws Exception
@@ -225,6 +239,63 @@ public void testPageSource()
assertEquals((int) stats.getLoadedBlockBytes().getAllTime().getCount(), 50);
}
+ @Test(dataProvider = "rowCount")
+ public void testMaxReadBytes(int rowCount)
+ throws Exception
+ {
+ int maxReadBytes = 1_000;
+ HiveClientConfig config = new HiveClientConfig();
+ config.setOrcMaxReadBlockSize(new DataSize(maxReadBytes, BYTE));
+ ConnectorSession session = new TestingConnectorSession(new HiveSessionProperties(config).getSessionProperties());
+ FileFormatDataSourceStats stats = new FileFormatDataSourceStats();
+
+ // Build a table where every row gets larger, so we can test that the "batchSize" reduces
+ int numColumns = 5;
+ int step = 250;
+ ImmutableList.Builder columnBuilder = ImmutableList.builder()
+ .add(new TestColumn("p_empty_string", javaStringObjectInspector, () -> "", true));
+ GrowingTestColumn[] dataColumns = new GrowingTestColumn[numColumns];
+ for (int i = 0; i < numColumns; i++) {
+ dataColumns[i] = new GrowingTestColumn("p_string", javaStringObjectInspector, () -> Long.toHexString(random.nextLong()), false, step * (i + 1));
+ columnBuilder.add(dataColumns[i]);
+ }
+ List testColumns = columnBuilder.build();
+ File tempFile = File.createTempFile("presto_test_orc_page_source_max_read_bytes", "orc");
+ tempFile.delete();
+
+ TestPreparer testPreparer = new TestPreparer(tempFile.getAbsolutePath(), testColumns, rowCount, rowCount);
+ ConnectorPageSource pageSource = testPreparer.newPageSource(stats, session);
+
+ try {
+ int positionCount = 0;
+ while (true) {
+ Page page = pageSource.getNextPage();
+ if (pageSource.isFinished()) {
+ break;
+ }
+ assertNotNull(page);
+ page.assureLoaded();
+ positionCount += page.getPositionCount();
+ // assert upper bound is tight
+ // ignore the first MAX_BATCH_SIZE rows given the sizes are set when loading the blocks
+ if (positionCount > MAX_BATCH_SIZE) {
+ // either the block is bounded by maxReadBytes or we just load one single large block
+ // an error margin MAX_BATCH_SIZE / step is needed given the block sizes are increasing
+ assertTrue(page.getSizeInBytes() < maxReadBytes * (MAX_BATCH_SIZE / step) || 1 == page.getPositionCount());
+ }
+ }
+
+ // verify the stats are correctly recorded
+ Distribution distribution = stats.getMaxCombinedBytesPerRow().getAllTime();
+ assertEquals((int) distribution.getCount(), 1);
+ assertEquals((int) distribution.getMax(), Arrays.stream(dataColumns).mapToInt(GrowingTestColumn::getMaxSize).sum());
+ pageSource.close();
+ }
+ finally {
+ tempFile.delete();
+ }
+ }
+
@Test
public void testTableScanOperator()
throws Exception
@@ -323,6 +394,12 @@ private class TestPreparer
public TestPreparer(String tempFilePath)
throws Exception
+ {
+ this(tempFilePath, testColumns, NUM_ROWS, STRIPE_ROWS);
+ }
+
+ public TestPreparer(String tempFilePath, List testColumns, int numRows, int stripeRows)
+ throws Exception
{
OrcSerde serde = new OrcSerde();
schema = new Properties();
@@ -359,10 +436,20 @@ public TestPreparer(String tempFilePath)
columns = columnsBuilder.build();
types = typesBuilder.build();
- fileSplit = createTestFile(tempFilePath, new OrcOutputFormat(), serde, null, testColumns, NUM_ROWS);
+ fileSplit = createTestFile(tempFilePath, new OrcOutputFormat(), serde, null, testColumns, numRows, stripeRows);
+ }
+
+ public ConnectorPageSource newPageSource()
+ {
+ return newPageSource(new FileFormatDataSourceStats(), SESSION);
}
public ConnectorPageSource newPageSource(FileFormatDataSourceStats stats)
+ {
+ return newPageSource(stats, SESSION);
+ }
+
+ public ConnectorPageSource newPageSource(FileFormatDataSourceStats stats, ConnectorSession session)
{
OrcPageSourceFactory orcPageSourceFactory = new OrcPageSourceFactory(TYPE_MANAGER, false, HDFS_ENVIRONMENT, stats);
return HivePageSourceProvider.createHivePageSource(
@@ -370,11 +457,12 @@ public ConnectorPageSource newPageSource(FileFormatDataSourceStats stats)
ImmutableSet.of(orcPageSourceFactory),
"test",
new Configuration(),
- SESSION,
+ session,
fileSplit.getPath(),
OptionalInt.empty(),
fileSplit.getStart(),
fileSplit.getLength(),
+ fileSplit.getLength(),
schema,
TupleDomain.all(),
columns,
@@ -387,7 +475,7 @@ public ConnectorPageSource newPageSource(FileFormatDataSourceStats stats)
public SourceOperator newTableScanOperator(DriverContext driverContext)
{
- ConnectorPageSource pageSource = newPageSource(new FileFormatDataSourceStats());
+ ConnectorPageSource pageSource = newPageSource();
SourceOperatorFactory sourceOperatorFactory = new TableScanOperatorFactory(
0,
new PlanNodeId("0"),
@@ -402,7 +490,7 @@ public SourceOperator newTableScanOperator(DriverContext driverContext)
public SourceOperator newScanFilterAndProjectOperator(DriverContext driverContext)
{
- ConnectorPageSource pageSource = newPageSource(new FileFormatDataSourceStats());
+ ConnectorPageSource pageSource = newPageSource();
ImmutableList.Builder projectionsBuilder = ImmutableList.builder();
for (int i = 0; i < types.size(); i++) {
projectionsBuilder.add(field(i, types.get(i)));
@@ -437,7 +525,8 @@ public static FileSplit createTestFile(String filePath,
@SuppressWarnings("deprecation") SerDe serDe,
String compressionCodec,
List testColumns,
- int numRows)
+ int numRows,
+ int stripeRows)
throws Exception
{
// filter out partition keys, which are not written to the file
@@ -477,7 +566,7 @@ public static FileSplit createTestFile(String filePath,
Writable record = serDe.serialize(row, objectInspector);
recordWriter.write(record);
- if (rowNumber % STRIPE_ROWS == STRIPE_ROWS - 1) {
+ if (rowNumber % stripeRows == stripeRows - 1) {
flushStripe(recordWriter);
}
}
@@ -541,7 +630,7 @@ private static Constructor extends RecordWriter> getOrcWriterConstructor()
}
}
- public static final class TestColumn
+ public static class TestColumn
{
private final String name;
private final ObjectInspector objectInspector;
@@ -592,4 +681,41 @@ public String toString()
return sb.toString();
}
}
+
+ public static final class GrowingTestColumn
+ extends TestColumn
+ {
+ private final Supplier writeValue;
+ private int counter;
+ private int step;
+ private int maxSize;
+
+ public GrowingTestColumn(String name, ObjectInspector objectInspector, Supplier writeValue, boolean partitionKey, int step)
+ {
+ super(name, objectInspector, writeValue, partitionKey);
+ this.writeValue = writeValue;
+ this.counter = step;
+ this.step = step;
+ }
+
+ @Override
+ public Object getWriteValue()
+ {
+ StringBuilder builder = new StringBuilder();
+ String source = writeValue.get();
+ for (int i = 0; i < counter / step; i++) {
+ builder.append(source);
+ }
+ counter++;
+ if (builder.length() > maxSize) {
+ maxSize = builder.length();
+ }
+ return builder.toString();
+ }
+
+ public int getMaxSize()
+ {
+ return maxSize;
+ }
+ }
}
diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/benchmark/FileFormat.java b/presto-hive/src/test/java/com/facebook/presto/hive/benchmark/FileFormat.java
index 703a6a376ca7..fb8e2eb6d594 100644
--- a/presto-hive/src/test/java/com/facebook/presto/hive/benchmark/FileFormat.java
+++ b/presto-hive/src/test/java/com/facebook/presto/hive/benchmark/FileFormat.java
@@ -360,6 +360,7 @@ private static ConnectorPageSource createPageSource(
new Path(targetFile.getAbsolutePath()),
0,
targetFile.length(),
+ targetFile.length(),
createSchema(format, columnNames, columnTypes),
columnHandles,
TupleDomain.all(),
@@ -392,6 +393,7 @@ private static ConnectorPageSource createPageSource(
new Path(targetFile.getAbsolutePath()),
0,
targetFile.length(),
+ targetFile.length(),
createSchema(format, columnNames, columnTypes),
columnHandles,
TupleDomain.all(),
diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/benchmark/HiveFileFormatBenchmark.java b/presto-hive/src/test/java/com/facebook/presto/hive/benchmark/HiveFileFormatBenchmark.java
index 6fd6a89eab3b..fb32e0b9aa15 100644
--- a/presto-hive/src/test/java/com/facebook/presto/hive/benchmark/HiveFileFormatBenchmark.java
+++ b/presto-hive/src/test/java/com/facebook/presto/hive/benchmark/HiveFileFormatBenchmark.java
@@ -23,9 +23,9 @@
import com.facebook.presto.spi.Page;
import com.facebook.presto.spi.PageBuilder;
import com.facebook.presto.spi.block.BlockBuilder;
+import com.facebook.presto.spi.type.ArrayType;
import com.facebook.presto.spi.type.Type;
import com.facebook.presto.testing.TestingConnectorSession;
-import com.facebook.presto.type.ArrayType;
import com.google.common.collect.ImmutableList;
import io.airlift.slice.Slices;
import io.airlift.tpch.OrderColumn;
diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/metastore/TestingHiveMetastore.java b/presto-hive/src/test/java/com/facebook/presto/hive/metastore/TestingHiveMetastore.java
index f1a2503f50a6..d92737c86b2f 100644
--- a/presto-hive/src/test/java/com/facebook/presto/hive/metastore/TestingHiveMetastore.java
+++ b/presto-hive/src/test/java/com/facebook/presto/hive/metastore/TestingHiveMetastore.java
@@ -47,6 +47,7 @@
import static com.facebook.presto.hive.metastore.Database.DEFAULT_DATABASE_NAME;
import static com.facebook.presto.hive.metastore.HivePrivilegeInfo.HivePrivilege.OWNERSHIP;
import static com.facebook.presto.hive.metastore.MetastoreUtil.makePartName;
+import static com.facebook.presto.hive.metastore.MetastoreUtil.verifyCanDropColumn;
import static com.facebook.presto.hive.metastore.PrincipalType.ROLE;
import static com.facebook.presto.hive.metastore.PrincipalType.USER;
import static com.facebook.presto.spi.StandardErrorCode.ALREADY_EXISTS;
@@ -353,6 +354,30 @@ public synchronized void renameColumn(String databaseName, String tableName, Str
relations.put(name, newTable);
}
+ @Override
+ public synchronized void dropColumn(String databaseName, String tableName, String columnName)
+ {
+ SchemaTableName name = new SchemaTableName(databaseName, tableName);
+ Table oldTable = getRequiredTable(name);
+
+ verifyCanDropColumn(this, databaseName, tableName, columnName);
+ if (!oldTable.getColumn(columnName).isPresent()) {
+ throw new ColumnNotFoundException(name, columnName);
+ }
+
+ ImmutableList.Builder newDataColumns = ImmutableList.builder();
+ for (Column fieldSchema : oldTable.getDataColumns()) {
+ if (!fieldSchema.getName().equals(columnName)) {
+ newDataColumns.add(fieldSchema);
+ }
+ }
+
+ Table newTable = Table.builder(oldTable)
+ .setDataColumns(newDataColumns.build())
+ .build();
+ relations.put(name, newTable);
+ }
+
@Override
public synchronized Optional> getAllTables(String databaseName)
{
diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/parquet/ParquetTester.java b/presto-hive/src/test/java/com/facebook/presto/hive/parquet/ParquetTester.java
index 939986159ed2..8d6943dd9b23 100644
--- a/presto-hive/src/test/java/com/facebook/presto/hive/parquet/ParquetTester.java
+++ b/presto-hive/src/test/java/com/facebook/presto/hive/parquet/ParquetTester.java
@@ -164,7 +164,7 @@ private static void assertFileContents(JobConf jobConf,
{
Path path = new Path(tempFile.getFile().toURI());
FileSystem fileSystem = path.getFileSystem(jobConf);
- ParquetMetadata parquetMetadata = ParquetMetadataReader.readFooter(fileSystem, path);
+ ParquetMetadata parquetMetadata = ParquetMetadataReader.readFooter(fileSystem, path, fileSystem.getFileStatus(path).getLen());
FileMetaData fileMetaData = parquetMetadata.getFileMetaData();
MessageType fileSchema = fileMetaData.getSchema();
diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/security/TestLegacySecurityConfig.java b/presto-hive/src/test/java/com/facebook/presto/hive/security/TestLegacySecurityConfig.java
index 3bc2bc8f11f1..0ed1a6896f90 100644
--- a/presto-hive/src/test/java/com/facebook/presto/hive/security/TestLegacySecurityConfig.java
+++ b/presto-hive/src/test/java/com/facebook/presto/hive/security/TestLegacySecurityConfig.java
@@ -29,6 +29,7 @@ public void testDefaults()
{
assertRecordedDefaults(recordDefaults(LegacySecurityConfig.class)
.setAllowAddColumn(false)
+ .setAllowDropColumn(false)
.setAllowDropTable(false)
.setAllowRenameTable(false)
.setAllowRenameColumn(false));
@@ -39,6 +40,7 @@ public void testExplicitPropertyMappings()
{
Map properties = new ImmutableMap.Builder()
.put("hive.allow-add-column", "true")
+ .put("hive.allow-drop-column", "true")
.put("hive.allow-drop-table", "true")
.put("hive.allow-rename-table", "true")
.put("hive.allow-rename-column", "true")
@@ -46,6 +48,7 @@ public void testExplicitPropertyMappings()
LegacySecurityConfig expected = new LegacySecurityConfig()
.setAllowAddColumn(true)
+ .setAllowDropColumn(true)
.setAllowDropTable(true)
.setAllowRenameTable(true)
.setAllowRenameColumn(true);
diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/util/TestSerDeUtils.java b/presto-hive/src/test/java/com/facebook/presto/hive/util/TestSerDeUtils.java
index 58d0fe7ea699..401bc6264610 100644
--- a/presto-hive/src/test/java/com/facebook/presto/hive/util/TestSerDeUtils.java
+++ b/presto-hive/src/test/java/com/facebook/presto/hive/util/TestSerDeUtils.java
@@ -18,8 +18,8 @@
import com.facebook.presto.spi.block.BlockBuilder;
import com.facebook.presto.spi.block.BlockBuilderStatus;
import com.facebook.presto.spi.block.InterleavedBlockBuilder;
-import com.facebook.presto.type.ArrayType;
-import com.facebook.presto.type.RowType;
+import com.facebook.presto.spi.type.ArrayType;
+import com.facebook.presto.spi.type.RowType;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.reflect.TypeToken;
diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml
index 7717cf7aa355..3012eb3044c1 100644
--- a/presto-jdbc/pom.xml
+++ b/presto-jdbc/pom.xml
@@ -5,7 +5,7 @@
com.facebook.presto
presto-root
- 0.179-tw-0.36
+ 0.181-tw-0.37
presto-jdbc
@@ -20,6 +20,12 @@
com.facebook.presto
presto-client
+
+
+ javax.validation
+ validation-api
+
+
@@ -28,39 +34,8 @@
- io.airlift
- http-client
-
-
-
- io.airlift
- configuration
-
-
-
- io.airlift
- trace-token
-
-
-
- com.google.inject
- guice
-
-
- com.google.inject.extensions
- guice-multibindings
-
-
-
- org.weakref
- jmxutils
-
-
-
- org.eclipse.jetty
- jetty-servlet
-
-
+ com.squareup.okhttp3
+ okhttp
@@ -72,10 +47,6 @@
io.airlift
json
-
- javax.inject
- javax.inject
-
com.google.inject
guice
@@ -97,11 +68,6 @@
guava
-
- com.google.code.findbugs
- annotations
-
-
com.facebook.presto
@@ -144,6 +110,12 @@
concurrent
test
+
+
+ com.squareup.okhttp3
+ mockwebserver
+ test
+
@@ -199,12 +171,12 @@
${shadeBase}.joda.time
- org.eclipse.jetty
- ${shadeBase}.jetty
+ okhttp3
+ ${shadeBase}.okhttp3
- org.HdrHistogram
- ${shadeBase}.HdrHistogram
+ okio
+ ${shadeBase}.okio
@@ -212,12 +184,8 @@
*:*
META-INF/maven/**
- META-INF/*.xml
- META-INF/services/org.eclipse.**
META-INF/services/com.fasterxml.**
LICENSE
- *.css
- *.html
@@ -226,10 +194,11 @@
**
+
- javax.validation:validation-api
+ com.squareup.okhttp3:okhttp
- **
+ publicsuffixes.gz
diff --git a/presto-jdbc/src/main/java/com/facebook/presto/jdbc/AbstractConnectionProperty.java b/presto-jdbc/src/main/java/com/facebook/presto/jdbc/AbstractConnectionProperty.java
new file mode 100644
index 000000000000..814a27e14f8d
--- /dev/null
+++ b/presto-jdbc/src/main/java/com/facebook/presto/jdbc/AbstractConnectionProperty.java
@@ -0,0 +1,166 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.facebook.presto.jdbc;
+
+import java.io.File;
+import java.sql.DriverPropertyInfo;
+import java.sql.SQLException;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.function.Predicate;
+
+import static java.lang.String.format;
+import static java.util.Locale.ENGLISH;
+import static java.util.Objects.requireNonNull;
+
+abstract class AbstractConnectionProperty
+ implements ConnectionProperty
+{
+ private final String key;
+ private final Optional defaultValue;
+ private final Predicate isRequired;
+ private final Predicate isAllowed;
+ private final Converter converter;
+
+ protected AbstractConnectionProperty(
+ String key,
+ Optional defaultValue,
+ Predicate isRequired,
+ Predicate isAllowed,
+ Converter converter)
+ {
+ this.key = requireNonNull(key, "key is null");
+ this.defaultValue = requireNonNull(defaultValue, "defaultValue is null");
+ this.isRequired = requireNonNull(isRequired, "isRequired is null");
+ this.isAllowed = requireNonNull(isAllowed, "isAllowed is null");
+ this.converter = requireNonNull(converter, "converter is null");
+ }
+
+ protected AbstractConnectionProperty(
+ String key,
+ Predicate required,
+ Predicate allowed,
+ Converter converter)
+ {
+ this(key, Optional.empty(), required, allowed, converter);
+ }
+
+ @Override
+ public String getKey()
+ {
+ return key;
+ }
+
+ @Override
+ public Optional getDefault()
+ {
+ return defaultValue;
+ }
+
+ @Override
+ public DriverPropertyInfo getDriverPropertyInfo(Properties mergedProperties)
+ {
+ String currentValue = mergedProperties.getProperty(key);
+ DriverPropertyInfo result = new DriverPropertyInfo(key, currentValue);
+ result.required = isRequired.test(mergedProperties);
+ return result;
+ }
+
+ @Override
+ public boolean isRequired(Properties properties)
+ {
+ return isRequired.test(properties);
+ }
+
+ @Override
+ public boolean isAllowed(Properties properties)
+ {
+ return !properties.containsKey(key) || isAllowed.test(properties);
+ }
+
+ @Override
+ public Optional getValue(Properties properties)
+ throws SQLException
+ {
+ String value = properties.getProperty(key);
+ if (value == null) {
+ if (isRequired(properties)) {
+ throw new SQLException(format("Connection property '%s' is required", key));
+ }
+ return Optional.empty();
+ }
+ if (value.isEmpty()) {
+ throw new SQLException(format("Connection property '%s' value is empty", key));
+ }
+
+ try {
+ return Optional.of(converter.convert(value));
+ }
+ catch (RuntimeException e) {
+ throw new SQLException(format("Connection property '%s' value is invalid: %s", key, value), e);
+ }
+ }
+
+ @Override
+ public void validate(Properties properties)
+ throws SQLException
+ {
+ if (!isAllowed(properties)) {
+ throw new SQLException(format("Connection property '%s' is not allowed", key));
+ }
+
+ getValue(properties);
+ }
+
+ protected static final Predicate REQUIRED = properties -> true;
+ protected static final Predicate NOT_REQUIRED = properties -> false;
+
+ protected static final Predicate ALLOWED = properties -> true;
+
+ interface Converter
+ {
+ T convert(String value);
+ }
+
+ protected static final Converter STRING_CONVERTER = value -> value;
+ protected static final Converter FILE_CONVERTER = File::new;
+
+ protected static final Converter BOOLEAN_CONVERTER = value -> {
+ switch (value.toLowerCase(ENGLISH)) {
+ case "true":
+ return true;
+ case "false":
+ return false;
+ }
+ throw new IllegalArgumentException("value must be 'true' or 'false'");
+ };
+
+ protected interface CheckedPredicate
+ {
+ boolean test(T t)
+ throws SQLException;
+ }
+
+ protected static Predicate checkedPredicate(CheckedPredicate predicate)
+ {
+ return t -> {
+ try {
+ return predicate.test(t);
+ }
+ catch (SQLException e) {
+ return false;
+ }
+ };
+ }
+}
diff --git a/presto-jdbc/src/main/java/com/facebook/presto/jdbc/ConnectionProperties.java b/presto-jdbc/src/main/java/com/facebook/presto/jdbc/ConnectionProperties.java
new file mode 100644
index 000000000000..8ed6f7a63ecc
--- /dev/null
+++ b/presto-jdbc/src/main/java/com/facebook/presto/jdbc/ConnectionProperties.java
@@ -0,0 +1,227 @@
+/*
+ * 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.jdbc;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.net.HostAndPort;
+
+import java.io.File;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.function.Predicate;
+
+import static com.facebook.presto.jdbc.AbstractConnectionProperty.checkedPredicate;
+import static java.util.Collections.unmodifiableMap;
+import static java.util.function.Function.identity;
+import static java.util.stream.Collectors.toMap;
+
+final class ConnectionProperties
+{
+ public static final ConnectionProperty USER = new User();
+ public static final ConnectionProperty PASSWORD = new Password();
+ public static final ConnectionProperty SOCKS_PROXY = new SocksProxy();
+ public static final ConnectionProperty HTTP_PROXY = new HttpProxy();
+ public static final ConnectionProperty SSL = new Ssl();
+ public static final ConnectionProperty SSL_TRUST_STORE_PATH = new SslTrustStorePath();
+ public static final ConnectionProperty SSL_TRUST_STORE_PASSWORD = new SslTrustStorePassword();
+ public static final ConnectionProperty KERBEROS_REMOTE_SERICE_NAME = new KerberosRemoteServiceName();
+ public static final ConnectionProperty KERBEROS_USE_CANONICAL_HOSTNAME = new KerberosUseCanonicalHostname();
+ public static final ConnectionProperty KERBEROS_PRINCIPAL = new KerberosPrincipal();
+ public static final ConnectionProperty KERBEROS_CONFIG_PATH = new KerberosConfigPath();
+ public static final ConnectionProperty KERBEROS_KEYTAB_PATH = new KerberosKeytabPath();
+ public static final ConnectionProperty KERBEROS_CREDENTIAL_CACHE_PATH = new KerberosCredentialCachePath();
+
+ private static final Set> ALL_PROPERTIES = ImmutableSet.>builder()
+ .add(USER)
+ .add(PASSWORD)
+ .add(SOCKS_PROXY)
+ .add(HTTP_PROXY)
+ .add(SSL)
+ .add(SSL_TRUST_STORE_PATH)
+ .add(SSL_TRUST_STORE_PASSWORD)
+ .add(KERBEROS_REMOTE_SERICE_NAME)
+ .add(KERBEROS_USE_CANONICAL_HOSTNAME)
+ .add(KERBEROS_PRINCIPAL)
+ .add(KERBEROS_CONFIG_PATH)
+ .add(KERBEROS_KEYTAB_PATH)
+ .add(KERBEROS_CREDENTIAL_CACHE_PATH)
+ .build();
+
+ private static final Map> KEY_LOOKUP = unmodifiableMap(ALL_PROPERTIES.stream()
+ .collect(toMap(ConnectionProperty::getKey, identity())));
+
+ private static final Map DEFAULTS;
+
+ static {
+ ImmutableMap.Builder defaults = ImmutableMap.builder();
+ for (ConnectionProperty> property : ALL_PROPERTIES) {
+ property.getDefault().ifPresent(value -> defaults.put(property.getKey(), value));
+ }
+ DEFAULTS = defaults.build();
+ }
+
+ private ConnectionProperties() {}
+
+ public static ConnectionProperty> forKey(String propertiesKey)
+ {
+ return KEY_LOOKUP.get(propertiesKey);
+ }
+
+ public static Set> allProperties()
+ {
+ return ALL_PROPERTIES;
+ }
+
+ public static Map getDefaults()
+ {
+ return DEFAULTS;
+ }
+
+ private static class User
+ extends AbstractConnectionProperty
+ {
+ public User()
+ {
+ super("user", REQUIRED, ALLOWED, STRING_CONVERTER);
+ }
+ }
+
+ private static class Password
+ extends AbstractConnectionProperty
+ {
+ public Password()
+ {
+ super("password", NOT_REQUIRED, ALLOWED, STRING_CONVERTER);
+ }
+ }
+
+ private static class SocksProxy
+ extends AbstractConnectionProperty
+ {
+ private static final Predicate NO_HTTP_PROXY =
+ checkedPredicate(properties -> !HTTP_PROXY.getValue(properties).isPresent());
+
+ public SocksProxy()
+ {
+ super("socksProxy", NOT_REQUIRED, NO_HTTP_PROXY, HostAndPort::fromString);
+ }
+ }
+
+ private static class HttpProxy
+ extends AbstractConnectionProperty
+ {
+ private static final Predicate NO_SOCKS_PROXY =
+ checkedPredicate(properties -> !SOCKS_PROXY.getValue(properties).isPresent());
+
+ public HttpProxy()
+ {
+ super("httpProxy", NOT_REQUIRED, NO_SOCKS_PROXY, HostAndPort::fromString);
+ }
+ }
+
+ private static class Ssl
+ extends AbstractConnectionProperty
+ {
+ public Ssl()
+ {
+ super("SSL", Optional.of("false"), NOT_REQUIRED, ALLOWED, BOOLEAN_CONVERTER);
+ }
+ }
+
+ private static class SslTrustStorePath
+ extends AbstractConnectionProperty
+ {
+ private static final Predicate IF_SSL_ENABLED =
+ checkedPredicate(properties -> SSL.getValue(properties).orElse(false));
+
+ public SslTrustStorePath()
+ {
+ super("SSLTrustStorePath", NOT_REQUIRED, IF_SSL_ENABLED, STRING_CONVERTER);
+ }
+ }
+
+ private static class SslTrustStorePassword
+ extends AbstractConnectionProperty
+ {
+ private static final Predicate IF_TRUST_STORE =
+ checkedPredicate(properties -> SSL_TRUST_STORE_PATH.getValue(properties).isPresent());
+
+ public SslTrustStorePassword()
+ {
+ super("SSLTrustStorePassword", NOT_REQUIRED, IF_TRUST_STORE, STRING_CONVERTER);
+ }
+ }
+
+ private static class KerberosRemoteServiceName
+ extends AbstractConnectionProperty
+ {
+ public KerberosRemoteServiceName()
+ {
+ super("KerberosRemoteServiceName", NOT_REQUIRED, ALLOWED, STRING_CONVERTER);
+ }
+ }
+
+ private static Predicate isKerberosEnabled()
+ {
+ return checkedPredicate(properties -> KERBEROS_REMOTE_SERICE_NAME.getValue(properties).isPresent());
+ }
+
+ private static class KerberosPrincipal
+ extends AbstractConnectionProperty
+ {
+ public KerberosPrincipal()
+ {
+ super("KerberosPrincipal", NOT_REQUIRED, isKerberosEnabled(), STRING_CONVERTER);
+ }
+ }
+
+ private static class KerberosUseCanonicalHostname
+ extends AbstractConnectionProperty
+ {
+ public KerberosUseCanonicalHostname()
+ {
+ super("KerberosUseCanonicalHostname", Optional.of("true"), isKerberosEnabled(), ALLOWED, BOOLEAN_CONVERTER);
+ }
+ }
+
+ private static class KerberosConfigPath
+ extends AbstractConnectionProperty
+ {
+ public KerberosConfigPath()
+ {
+ super("KerberosConfigPath", NOT_REQUIRED, isKerberosEnabled(), FILE_CONVERTER);
+ }
+ }
+
+ private static class KerberosKeytabPath
+ extends AbstractConnectionProperty
+ {
+ public KerberosKeytabPath()
+ {
+ super("KerberosKeytabPath", NOT_REQUIRED, isKerberosEnabled(), FILE_CONVERTER);
+ }
+ }
+
+ private static class KerberosCredentialCachePath
+ extends AbstractConnectionProperty
+ {
+ public KerberosCredentialCachePath()
+ {
+ super("KerberosCredentialCachePath", NOT_REQUIRED, isKerberosEnabled(), FILE_CONVERTER);
+ }
+ }
+}
diff --git a/presto-jdbc/src/main/java/com/facebook/presto/jdbc/ConnectionProperty.java b/presto-jdbc/src/main/java/com/facebook/presto/jdbc/ConnectionProperty.java
new file mode 100644
index 000000000000..ac2e90897803
--- /dev/null
+++ b/presto-jdbc/src/main/java/com/facebook/presto/jdbc/ConnectionProperty.java
@@ -0,0 +1,47 @@
+/*
+ * 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.jdbc;
+
+import java.sql.DriverPropertyInfo;
+import java.sql.SQLException;
+import java.util.Optional;
+import java.util.Properties;
+
+import static java.lang.String.format;
+
+interface ConnectionProperty
+{
+ String getKey();
+
+ Optional getDefault();
+
+ DriverPropertyInfo getDriverPropertyInfo(Properties properties);
+
+ boolean isRequired(Properties properties);
+
+ boolean isAllowed(Properties properties);
+
+ Optional getValue(Properties properties)
+ throws SQLException;
+
+ default T getRequiredValue(Properties properties)
+ throws SQLException
+ {
+ return getValue(properties).orElseThrow(() ->
+ new SQLException(format("Connection property '%s' is required", getKey())));
+ }
+
+ void validate(Properties properties)
+ throws SQLException;
+}
diff --git a/presto-jdbc/src/main/java/com/facebook/presto/jdbc/PrestoConnection.java b/presto-jdbc/src/main/java/com/facebook/presto/jdbc/PrestoConnection.java
index 5f62ed93c7be..4e54ac4f3b39 100644
--- a/presto-jdbc/src/main/java/com/facebook/presto/jdbc/PrestoConnection.java
+++ b/presto-jdbc/src/main/java/com/facebook/presto/jdbc/PrestoConnection.java
@@ -73,7 +73,7 @@ public class PrestoConnection
private final AtomicReference